├── LICENSE ├── SUMMARY.adoc ├── book-intro.adoc ├── book.json ├── graffles ├── kafka-streams-AssignedStandbyTasks.graffle ├── kafka-streams-AssignedTasks-addNewTask.graffle ├── kafka-streams-AssignedTasks-initializeNewTasks.graffle ├── kafka-streams-InternalStreamsBuilder.graffle ├── kafka-streams-InternalTopologyBuilder-Topology.graffle ├── kafka-streams-InternalTopologyBuilder-build.graffle ├── kafka-streams-KafkaStreams.graffle ├── kafka-streams-RebalanceListener-creating-instance.graffle ├── kafka-streams-RebalanceListener.graffle ├── kafka-streams-RecordQueue.graffle ├── kafka-streams-StoreChangelogReader.graffle ├── kafka-streams-StreamTask-addRecords.graffle ├── kafka-streams-StreamTask-process.graffle ├── kafka-streams-StreamThread-TASK_MANAGER_FOR_PARTITION_ASSIGNOR.graffle ├── kafka-streams-StreamThread-addRecordsToTasks.graffle ├── kafka-streams-StreamThread-create.graffle ├── kafka-streams-StreamThread-lifecycle.graffle ├── kafka-streams-StreamThread-runLoop.graffle ├── kafka-streams-StreamThread-runOnce.graffle ├── kafka-streams-StreamThread.graffle ├── kafka-streams-StreamsBuilder.graffle ├── kafka-streams-StreamsMetadataState-onChange.graffle ├── kafka-streams-StreamsPartitionAssignor-onAssignment.graffle ├── kafka-streams-TaskManager-addStandbyTasks.graffle ├── kafka-streams-TaskManager-addStreamTasks.graffle ├── kafka-streams-TaskManager-createTasks.graffle ├── kafka-streams-TaskManager-hasActiveRunningTasks.graffle ├── kafka-streams-TaskManager-hasStandbyRunningTasks.graffle ├── kafka-streams-TaskManager-updateNewAndRestoringTasks.graffle ├── kafka-streams-TaskManager.graffle ├── kafka-streams-Topology-InternalTopologyBuilder.graffle ├── kafka-streams-WindowBytesStoreSupplier.graffle └── kafka-streams-WindowStoreBuilder.graffle ├── images ├── kafka-streams-AssignedStandbyTasks.png ├── kafka-streams-AssignedTasks-addNewTask.png ├── kafka-streams-AssignedTasks-initializeNewTasks.png ├── kafka-streams-InternalStreamsBuilder.png ├── kafka-streams-InternalTopologyBuilder-Topology.png ├── kafka-streams-InternalTopologyBuilder-build.png ├── kafka-streams-KafkaStreams.png ├── kafka-streams-RebalanceListener-creating-instance.png ├── kafka-streams-RebalanceListener.png ├── kafka-streams-RecordQueue.png ├── kafka-streams-StoreChangelogReader.png ├── kafka-streams-StreamTask-addRecords.png ├── kafka-streams-StreamTask-process.png ├── kafka-streams-StreamThread-TASK_MANAGER_FOR_PARTITION_ASSIGNOR.png ├── kafka-streams-StreamThread-addRecordsToTasks.png ├── kafka-streams-StreamThread-create.png ├── kafka-streams-StreamThread-lifecycle.png ├── kafka-streams-StreamThread-runLoop.png ├── kafka-streams-StreamThread-runOnce.png ├── kafka-streams-StreamThread.png ├── kafka-streams-StreamsBuilder.png ├── kafka-streams-StreamsMetadataState-onChange.png ├── kafka-streams-StreamsPartitionAssignor-onAssignment.png ├── kafka-streams-TaskManager-addStandbyTasks.png ├── kafka-streams-TaskManager-addStreamTasks.png ├── kafka-streams-TaskManager-createTasks.png ├── kafka-streams-TaskManager-hasActiveRunningTasks.png ├── kafka-streams-TaskManager-hasStandbyRunningTasks.png ├── kafka-streams-TaskManager-updateNewAndRestoringTasks.png ├── kafka-streams-TaskManager.png ├── kafka-streams-Topology-InternalTopologyBuilder.png ├── kafka-streams-WindowBytesStoreSupplier.png └── kafka-streams-WindowStoreBuilder.png ├── kafka-logging.adoc ├── kafka-streams-AbstractNotifyingBatchingRestoreCallback.adoc ├── kafka-streams-AbstractNotifyingRestoreCallback.adoc ├── kafka-streams-AbstractProcessor.adoc ├── kafka-streams-BatchingStateRestoreCallback.adoc ├── kafka-streams-Cancellable.adoc ├── kafka-streams-Consumed.adoc ├── kafka-streams-DefaultPartitionGrouper.adoc ├── kafka-streams-DelegatingStateRestoreListener.adoc ├── kafka-streams-ExtractRecordMetadataTimestamp.adoc ├── kafka-streams-FailOnInvalidTimestamp.adoc ├── kafka-streams-GlobalKTable.adoc ├── kafka-streams-Grouped.adoc ├── kafka-streams-JoinWindows.adoc ├── kafka-streams-Joined.adoc ├── kafka-streams-KGroupedStream.adoc ├── kafka-streams-KGroupedTable.adoc ├── kafka-streams-KStream.adoc ├── kafka-streams-KTable.adoc ├── kafka-streams-KafkaClientSupplier.adoc ├── kafka-streams-KafkaStreams.adoc ├── kafka-streams-KeyValueBytesStoreSupplier.adoc ├── kafka-streams-KeyValueMapper.adoc ├── kafka-streams-KeyValueStore.adoc ├── kafka-streams-Materialized.adoc ├── kafka-streams-PartitionGrouper.adoc ├── kafka-streams-Printed.adoc ├── kafka-streams-Processor.adoc ├── kafka-streams-ProcessorContext.adoc ├── kafka-streams-ProcessorSupplier.adoc ├── kafka-streams-Produced.adoc ├── kafka-streams-PunctuationSchedule.adoc ├── kafka-streams-Punctuator.adoc ├── kafka-streams-ReadOnlyKeyValueStore.adoc ├── kafka-streams-RecordContext.adoc ├── kafka-streams-SessionBytesStoreSupplier.adoc ├── kafka-streams-SessionStore.adoc ├── kafka-streams-SessionWindowedKStream.adoc ├── kafka-streams-Stamped.adoc ├── kafka-streams-StateRestoreCallback.adoc ├── kafka-streams-StateRestoreListener.adoc ├── kafka-streams-StateStore.adoc ├── kafka-streams-StoreBuilder.adoc ├── kafka-streams-StoreSupplier.adoc ├── kafka-streams-Stores.adoc ├── kafka-streams-StreamPartitioner.adoc ├── kafka-streams-StreamTask-TaskMetrics.adoc ├── kafka-streams-StreamThread-RebalanceListener.adoc ├── kafka-streams-StreamThread-StateListener.adoc ├── kafka-streams-StreamThreads-StreamTasks-and-StandbyTasks.adoc ├── kafka-streams-StreamsBuilder.adoc ├── kafka-streams-StreamsConfig.adoc ├── kafka-streams-StreamsMetrics.adoc ├── kafka-streams-TaskId.adoc ├── kafka-streams-TimeWindowedKStream.adoc ├── kafka-streams-TimeWindowedSerializer.adoc ├── kafka-streams-TimeWindows.adoc ├── kafka-streams-TimestampExtractor.adoc ├── kafka-streams-TimestampedKeyValueStore.adoc ├── kafka-streams-To.adoc ├── kafka-streams-TopicNameExtractor.adoc ├── kafka-streams-Topology.adoc ├── kafka-streams-TopologyDescription-Node.adoc ├── kafka-streams-TopologyDescription-Processor.adoc ├── kafka-streams-TopologyDescription-Sink.adoc ├── kafka-streams-TopologyDescription-Source.adoc ├── kafka-streams-TopologyDescription.adoc ├── kafka-streams-TopologyTestDriver.adoc ├── kafka-streams-Transformer.adoc ├── kafka-streams-TransformerSupplier.adoc ├── kafka-streams-UnlimitedWindows.adoc ├── kafka-streams-ValueJoiner.adoc ├── kafka-streams-ValueTransformer.adoc ├── kafka-streams-ValueTransformerSupplier.adoc ├── kafka-streams-ValueTransformerWithKey.adoc ├── kafka-streams-WallclockTimestampExtractor.adoc ├── kafka-streams-Window.adoc ├── kafka-streams-WindowBytesStoreSupplier.adoc ├── kafka-streams-WindowStore.adoc ├── kafka-streams-Windowed.adoc ├── kafka-streams-WindowedSerdes.adoc ├── kafka-streams-Windows.adoc ├── kafka-streams-demo-creating-topology-with-state-store-logging-enabled.adoc ├── kafka-streams-exactly-once-support-eos.adoc ├── kafka-streams-internals-AbstractProcessorContext.adoc ├── kafka-streams-internals-AbstractRocksDBSegmentedBytesStore.adoc ├── kafka-streams-internals-AbstractStateManager.adoc ├── kafka-streams-internals-AbstractStoreBuilder.adoc ├── kafka-streams-internals-AbstractStream.adoc ├── kafka-streams-internals-AbstractTask.adoc ├── kafka-streams-internals-AbstractTaskCreator.adoc ├── kafka-streams-internals-AssignedStandbyTasks.adoc ├── kafka-streams-internals-AssignedStreamsTasks.adoc ├── kafka-streams-internals-AssignedTasks.adoc ├── kafka-streams-internals-AssignmentInfo.adoc ├── kafka-streams-internals-BaseJoinProcessorNode.adoc ├── kafka-streams-internals-BaseRepartitionNode.adoc ├── kafka-streams-internals-CacheFlushListener.adoc ├── kafka-streams-internals-CachedStateStore.adoc ├── kafka-streams-internals-CachingKeyValueStore.adoc ├── kafka-streams-internals-CachingSessionStore.adoc ├── kafka-streams-internals-CachingWindowStore.adoc ├── kafka-streams-internals-ChangeLoggingKeyValueBytesStore.adoc ├── kafka-streams-internals-ChangeLoggingSessionBytesStore.adoc ├── kafka-streams-internals-ChangeLoggingWindowBytesStore.adoc ├── kafka-streams-internals-ChangelogReader.adoc ├── kafka-streams-internals-Checkpointable.adoc ├── kafka-streams-internals-ClientMetadata.adoc ├── kafka-streams-internals-CompositeRestoreListener.adoc ├── kafka-streams-internals-ConsumedInternal.adoc ├── kafka-streams-internals-CopartitionedTopicsValidator.adoc ├── kafka-streams-internals-DefaultKafkaClientSupplier.adoc ├── kafka-streams-internals-ForwardingDisabledProcessorContext.adoc ├── kafka-streams-internals-GlobalKTableImpl.adoc ├── kafka-streams-internals-GlobalProcessorContextImpl.adoc ├── kafka-streams-internals-GlobalStateMaintainer.adoc ├── kafka-streams-internals-GlobalStateManager.adoc ├── kafka-streams-internals-GlobalStateManagerImpl.adoc ├── kafka-streams-internals-GlobalStateStoreProvider.adoc ├── kafka-streams-internals-GlobalStateUpdateTask.adoc ├── kafka-streams-internals-GlobalStoreNode.adoc ├── kafka-streams-internals-GlobalStreamThread.adoc ├── kafka-streams-internals-GroupedStreamAggregateBuilder.adoc ├── kafka-streams-internals-GroupedTableOperationRepartitionNode.adoc ├── kafka-streams-internals-GroupedTableOperationRepartitionNodeBuilder.adoc ├── kafka-streams-internals-InMemoryKeyValueStore.adoc ├── kafka-streams-internals-InMemorySessionBytesStoreSupplier.adoc ├── kafka-streams-internals-InMemorySessionStore.adoc ├── kafka-streams-internals-InMemoryTimeOrderedKeyValueBuffer.adoc ├── kafka-streams-internals-InMemoryWindowStore.adoc ├── kafka-streams-internals-InternalNameProvider.adoc ├── kafka-streams-internals-InternalProcessorContext.adoc ├── kafka-streams-internals-InternalStreamsBuilder.adoc ├── kafka-streams-internals-InternalTopicConfig.adoc ├── kafka-streams-internals-InternalTopicManager.adoc ├── kafka-streams-internals-InternalTopologyBuilder-AbstractNode.adoc ├── kafka-streams-internals-InternalTopologyBuilder-GlobalStore.adoc ├── kafka-streams-internals-InternalTopologyBuilder-NodeFactory.adoc ├── kafka-streams-internals-InternalTopologyBuilder-Processor.adoc ├── kafka-streams-internals-InternalTopologyBuilder-ProcessorNodeFactory.adoc ├── kafka-streams-internals-InternalTopologyBuilder-Sink.adoc ├── kafka-streams-internals-InternalTopologyBuilder-SinkNodeFactory.adoc ├── kafka-streams-internals-InternalTopologyBuilder-Source.adoc ├── kafka-streams-internals-InternalTopologyBuilder-SourceNodeFactory.adoc ├── kafka-streams-internals-InternalTopologyBuilder-StateStoreFactory.adoc ├── kafka-streams-internals-InternalTopologyBuilder-TopologyDescription.adoc ├── kafka-streams-internals-InternalTopologyBuilder.adoc ├── kafka-streams-internals-KGroupedStreamImpl.adoc ├── kafka-streams-internals-KGroupedTableImpl.adoc ├── kafka-streams-internals-KStreamAggProcessorSupplier.adoc ├── kafka-streams-internals-KStreamAggregate.adoc ├── kafka-streams-internals-KStreamBranch.adoc ├── kafka-streams-internals-KStreamFilter.adoc ├── kafka-streams-internals-KStreamFilterProcessor.adoc ├── kafka-streams-internals-KStreamImpl-KStreamImplJoin.adoc ├── kafka-streams-internals-KStreamImpl.adoc ├── kafka-streams-internals-KStreamJoinWindowProcessor.adoc ├── kafka-streams-internals-KStreamPassThrough.adoc ├── kafka-streams-internals-KStreamPeek.adoc ├── kafka-streams-internals-KStreamSessionWindowAggregate.adoc ├── kafka-streams-internals-KStreamSessionWindowAggregateProcessor.adoc ├── kafka-streams-internals-KStreamTransform.adoc ├── kafka-streams-internals-KStreamTransformProcessor.adoc ├── kafka-streams-internals-KStreamTransformValues.adoc ├── kafka-streams-internals-KStreamTransformValuesProcessor.adoc ├── kafka-streams-internals-KStreamWindowAggregate.adoc ├── kafka-streams-internals-KStreamWindowAggregateProcessor.adoc ├── kafka-streams-internals-KTableImpl.adoc ├── kafka-streams-internals-KTableKTableAbstractJoinValueGetterSupplier.adoc ├── kafka-streams-internals-KTableKTableJoinNode.adoc ├── kafka-streams-internals-KTableMaterializedValueGetterSupplier.adoc ├── kafka-streams-internals-KTableSource.adoc ├── kafka-streams-internals-KTableSourceProcessor.adoc ├── kafka-streams-internals-KTableSourceValueGetterSupplier.adoc ├── kafka-streams-internals-KTableSuppressProcessor.adoc ├── kafka-streams-internals-KTableValueGetter.adoc ├── kafka-streams-internals-KTableValueGetterSupplier.adoc ├── kafka-streams-internals-KeyValueStoreBuilder.adoc ├── kafka-streams-internals-KeyValueStoreMaterializer.adoc ├── kafka-streams-internals-KeyValueToTimestampedKeyValueByteStoreAdapter.adoc ├── kafka-streams-internals-MaterializedInternal.adoc ├── kafka-streams-internals-MemoryLRUCache.adoc ├── kafka-streams-internals-MeteredKeyValueBytesStore.adoc ├── kafka-streams-internals-MeteredKeyValueStore.adoc ├── kafka-streams-internals-MeteredSessionStore.adoc ├── kafka-streams-internals-MeteredWindowStore.adoc ├── kafka-streams-internals-NamedCache.adoc ├── kafka-streams-internals-NodeMetrics.adoc ├── kafka-streams-internals-OffsetCheckpoint.adoc ├── kafka-streams-internals-OptimizableRepartitionNode.adoc ├── kafka-streams-internals-PartitionGroup.adoc ├── kafka-streams-internals-ProcessorContextImpl.adoc ├── kafka-streams-internals-ProcessorGraphNode.adoc ├── kafka-streams-internals-ProcessorNode.adoc ├── kafka-streams-internals-ProcessorNodePunctuator.adoc ├── kafka-streams-internals-ProcessorParameters.adoc ├── kafka-streams-internals-ProcessorRecordContext.adoc ├── kafka-streams-internals-ProcessorStateManager.adoc ├── kafka-streams-internals-ProcessorTopology.adoc ├── kafka-streams-internals-ProducedInternal.adoc ├── kafka-streams-internals-ProducerSupplier.adoc ├── kafka-streams-internals-PunctuationQueue.adoc ├── kafka-streams-internals-QueryableStoreProvider.adoc ├── kafka-streams-internals-QuickUnion.adoc ├── kafka-streams-internals-RecordBatchingStateRestoreCallback.adoc ├── kafka-streams-internals-RecordCollector.adoc ├── kafka-streams-internals-RecordCollectorImpl.adoc ├── kafka-streams-internals-RecordDeserializer.adoc ├── kafka-streams-internals-RecordInfo.adoc ├── kafka-streams-internals-RecordQueue.adoc ├── kafka-streams-internals-RestoringTasks.adoc ├── kafka-streams-internals-RocksDBSegmentedBytesStore.adoc ├── kafka-streams-internals-RocksDBSessionStore.adoc ├── kafka-streams-internals-RocksDBStore.adoc ├── kafka-streams-internals-RocksDBTimestampedStore.adoc ├── kafka-streams-internals-RocksDBWindowStore.adoc ├── kafka-streams-internals-RocksDbKeyValueBytesStoreSupplier.adoc ├── kafka-streams-internals-RocksDbSessionBytesStoreSupplier.adoc ├── kafka-streams-internals-RocksDbWindowBytesStoreSupplier.adoc ├── kafka-streams-internals-SegmentedBytesStore.adoc ├── kafka-streams-internals-SessionStoreBuilder.adoc ├── kafka-streams-internals-SessionWindow.adoc ├── kafka-streams-internals-SessionWindowedKStreamImpl.adoc ├── kafka-streams-internals-SinkNode.adoc ├── kafka-streams-internals-SourceNode.adoc ├── kafka-streams-internals-StampedRecord.adoc ├── kafka-streams-internals-StandbyContextImpl.adoc ├── kafka-streams-internals-StandbyTask.adoc ├── kafka-streams-internals-StandbyTaskCreator.adoc ├── kafka-streams-internals-StateConsumer.adoc ├── kafka-streams-internals-StateDirectory.adoc ├── kafka-streams-internals-StateListener.adoc ├── kafka-streams-internals-StateManager.adoc ├── kafka-streams-internals-StateRestoreCallbackAdapter.adoc ├── kafka-streams-internals-StateRestorer.adoc ├── kafka-streams-internals-StateStoreNode.adoc ├── kafka-streams-internals-StateStoreProvider.adoc ├── kafka-streams-internals-StatefulProcessorNode.adoc ├── kafka-streams-internals-StaticTopicNameExtractor.adoc ├── kafka-streams-internals-StickyTaskAssignor.adoc ├── kafka-streams-internals-StoreChangeLogger.adoc ├── kafka-streams-internals-StoreChangelogReader.adoc ├── kafka-streams-internals-StreamSinkNode.adoc ├── kafka-streams-internals-StreamSourceNode.adoc ├── kafka-streams-internals-StreamStreamJoinNode.adoc ├── kafka-streams-internals-StreamTableJoinNode.adoc ├── kafka-streams-internals-StreamTask.adoc ├── kafka-streams-internals-StreamThread.adoc ├── kafka-streams-internals-StreamThreadStateStoreProvider.adoc ├── kafka-streams-internals-StreamsGraphNode.adoc ├── kafka-streams-internals-StreamsMetadataState.adoc ├── kafka-streams-internals-StreamsMetricsImpl.adoc ├── kafka-streams-internals-StreamsMetricsThreadImpl.adoc ├── kafka-streams-internals-StreamsPartitionAssignor.adoc ├── kafka-streams-internals-SubscriptionInfo.adoc ├── kafka-streams-internals-TableProcessorNode.adoc ├── kafka-streams-internals-TableSourceNode.adoc ├── kafka-streams-internals-Task.adoc ├── kafka-streams-internals-TaskAssignor.adoc ├── kafka-streams-internals-TaskCreator.adoc ├── kafka-streams-internals-TaskManager.adoc ├── kafka-streams-internals-ThreadCache.adoc ├── kafka-streams-internals-TimeOrderedKeyValueBuffer.adoc ├── kafka-streams-internals-TimeWindow.adoc ├── kafka-streams-internals-TimeWindowedKStreamImpl.adoc ├── kafka-streams-internals-TimestampedKeyValueStoreBuilder.adoc ├── kafka-streams-internals-TimestampedWindowStoreBuilder.adoc ├── kafka-streams-internals-TopicsInfo.adoc ├── kafka-streams-internals-UnlimitedWindow.adoc ├── kafka-streams-internals-UnwindowedChangelogTopicConfig.adoc ├── kafka-streams-internals-WindowStoreBuilder.adoc ├── kafka-streams-internals-WindowToTimestampedWindowByteStoreAdapter.adoc ├── kafka-streams-internals-WindowedChangelogTopicConfig.adoc ├── kafka-streams-internals-WindowedSerializer.adoc ├── kafka-streams-internals-WindowedStreamPartitioner.adoc ├── kafka-streams-internals-WrappedStateStore.adoc ├── kafka-streams-internals-WrappingStoreProvider.adoc ├── kafka-streams-multi-instance-kafka-streams-application.adoc ├── kafka-streams-processor-api.adoc ├── kafka-streams-properties.adoc ├── kafka-streams-scala-Consumed.adoc ├── kafka-streams-scala-Grouped.adoc ├── kafka-streams-scala-ImplicitConversions.adoc ├── kafka-streams-scala-Materialized.adoc ├── kafka-streams-scala-Produced.adoc ├── kafka-streams-scala-Serdes.adoc ├── kafka-streams-scala.adoc ├── kafka-streams-stateful-stream-processing.adoc ├── kafka-streams-streams-dsl.adoc ├── kafka-streams-why.adoc └── kafka-streams.adoc /book-intro.adoc: -------------------------------------------------------------------------------- 1 | == The Internals of Kafka Streams {{ book.kafka_version }} 2 | 3 | Welcome to *The Internals of Kafka Streams* gitbook! I'm very excited to have you here and hope you will enjoy exploring the internals of Kafka Streams as much as I have. 4 | 5 | [quote, Flannery O'Connor] 6 | I write to discover what I know. 7 | 8 | I'm https://pl.linkedin.com/in/jaceklaskowski[Jacek Laskowski], a freelance IT consultant, software engineer and technical instructor specializing in https://spark.apache.org/[Apache Spark], https://kafka.apache.org/[Apache Kafka] and https://kafka.apache.org/documentation/streams/[Kafka Streams] (with https://www.scala-lang.org/[Scala] and https://www.scala-sbt.org/[sbt]). 9 | 10 | I offer software development and consultancy services with hands-on in-depth workshops and mentoring. Reach out to me at jacek@japila.pl or https://twitter.com/jaceklaskowski[@jaceklaskowski] to discuss opportunities. 11 | 12 | Consider joining me at http://www.meetup.com/WarsawScala/[Warsaw Scala Enthusiasts] and http://www.meetup.com/Warsaw-Spark[Warsaw Spark] meetups in Warsaw, Poland. 13 | 14 | TIP: I'm also writing other books in the "The Internals of" series about https://bit.ly/apache-kafka-internals[Apache Kafka], http://books.japila.pl/apache-spark-internals/[Apache Spark], https://bit.ly/spark-sql-internals[Spark SQL], and https://bit.ly/spark-structured-streaming[Spark Structured Streaming]. 15 | 16 | Expect text and code snippets from a variety of public sources. Attribution follows. 17 | 18 | Now, let me introduce you to <>. 19 | -------------------------------------------------------------------------------- /book.json: -------------------------------------------------------------------------------- 1 | { 2 | "structure": { 3 | "readme": "book-intro.adoc" 4 | }, 5 | "plugins": ["ga"], 6 | "pluginsConfig": { 7 | "ga": { 8 | "token": "UA-86782445-5" 9 | } 10 | }, 11 | "variables": { 12 | "kafka_version": "2.3.0" 13 | } 14 | } 15 | -------------------------------------------------------------------------------- /graffles/kafka-streams-AssignedStandbyTasks.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-AssignedStandbyTasks.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-AssignedTasks-addNewTask.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-AssignedTasks-addNewTask.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-AssignedTasks-initializeNewTasks.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-AssignedTasks-initializeNewTasks.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-InternalStreamsBuilder.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-InternalStreamsBuilder.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-InternalTopologyBuilder-Topology.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-InternalTopologyBuilder-Topology.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-InternalTopologyBuilder-build.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-InternalTopologyBuilder-build.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-KafkaStreams.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-KafkaStreams.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-RebalanceListener-creating-instance.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-RebalanceListener-creating-instance.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-RebalanceListener.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-RebalanceListener.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-RecordQueue.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-RecordQueue.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-StoreChangelogReader.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-StoreChangelogReader.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-StreamTask-addRecords.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-StreamTask-addRecords.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-StreamTask-process.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-StreamTask-process.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-StreamThread-TASK_MANAGER_FOR_PARTITION_ASSIGNOR.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-StreamThread-TASK_MANAGER_FOR_PARTITION_ASSIGNOR.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-StreamThread-addRecordsToTasks.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-StreamThread-addRecordsToTasks.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-StreamThread-create.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-StreamThread-create.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-StreamThread-lifecycle.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-StreamThread-lifecycle.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-StreamThread-runLoop.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-StreamThread-runLoop.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-StreamThread-runOnce.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-StreamThread-runOnce.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-StreamThread.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-StreamThread.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-StreamsBuilder.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-StreamsBuilder.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-StreamsMetadataState-onChange.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-StreamsMetadataState-onChange.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-StreamsPartitionAssignor-onAssignment.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-StreamsPartitionAssignor-onAssignment.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-TaskManager-addStandbyTasks.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-TaskManager-addStandbyTasks.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-TaskManager-addStreamTasks.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-TaskManager-addStreamTasks.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-TaskManager-createTasks.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-TaskManager-createTasks.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-TaskManager-hasActiveRunningTasks.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-TaskManager-hasActiveRunningTasks.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-TaskManager-hasStandbyRunningTasks.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-TaskManager-hasStandbyRunningTasks.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-TaskManager-updateNewAndRestoringTasks.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-TaskManager-updateNewAndRestoringTasks.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-TaskManager.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-TaskManager.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-Topology-InternalTopologyBuilder.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-Topology-InternalTopologyBuilder.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-WindowBytesStoreSupplier.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-WindowBytesStoreSupplier.graffle -------------------------------------------------------------------------------- /graffles/kafka-streams-WindowStoreBuilder.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/graffles/kafka-streams-WindowStoreBuilder.graffle -------------------------------------------------------------------------------- /images/kafka-streams-AssignedStandbyTasks.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-AssignedStandbyTasks.png -------------------------------------------------------------------------------- /images/kafka-streams-AssignedTasks-addNewTask.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-AssignedTasks-addNewTask.png -------------------------------------------------------------------------------- /images/kafka-streams-AssignedTasks-initializeNewTasks.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-AssignedTasks-initializeNewTasks.png -------------------------------------------------------------------------------- /images/kafka-streams-InternalStreamsBuilder.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-InternalStreamsBuilder.png -------------------------------------------------------------------------------- /images/kafka-streams-InternalTopologyBuilder-Topology.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-InternalTopologyBuilder-Topology.png -------------------------------------------------------------------------------- /images/kafka-streams-InternalTopologyBuilder-build.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-InternalTopologyBuilder-build.png -------------------------------------------------------------------------------- /images/kafka-streams-KafkaStreams.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-KafkaStreams.png -------------------------------------------------------------------------------- /images/kafka-streams-RebalanceListener-creating-instance.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-RebalanceListener-creating-instance.png -------------------------------------------------------------------------------- /images/kafka-streams-RebalanceListener.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-RebalanceListener.png -------------------------------------------------------------------------------- /images/kafka-streams-RecordQueue.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-RecordQueue.png -------------------------------------------------------------------------------- /images/kafka-streams-StoreChangelogReader.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-StoreChangelogReader.png -------------------------------------------------------------------------------- /images/kafka-streams-StreamTask-addRecords.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-StreamTask-addRecords.png -------------------------------------------------------------------------------- /images/kafka-streams-StreamTask-process.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-StreamTask-process.png -------------------------------------------------------------------------------- /images/kafka-streams-StreamThread-TASK_MANAGER_FOR_PARTITION_ASSIGNOR.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-StreamThread-TASK_MANAGER_FOR_PARTITION_ASSIGNOR.png -------------------------------------------------------------------------------- /images/kafka-streams-StreamThread-addRecordsToTasks.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-StreamThread-addRecordsToTasks.png -------------------------------------------------------------------------------- /images/kafka-streams-StreamThread-create.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-StreamThread-create.png -------------------------------------------------------------------------------- /images/kafka-streams-StreamThread-lifecycle.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-StreamThread-lifecycle.png -------------------------------------------------------------------------------- /images/kafka-streams-StreamThread-runLoop.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-StreamThread-runLoop.png -------------------------------------------------------------------------------- /images/kafka-streams-StreamThread-runOnce.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-StreamThread-runOnce.png -------------------------------------------------------------------------------- /images/kafka-streams-StreamThread.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-StreamThread.png -------------------------------------------------------------------------------- /images/kafka-streams-StreamsBuilder.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-StreamsBuilder.png -------------------------------------------------------------------------------- /images/kafka-streams-StreamsMetadataState-onChange.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-StreamsMetadataState-onChange.png -------------------------------------------------------------------------------- /images/kafka-streams-StreamsPartitionAssignor-onAssignment.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-StreamsPartitionAssignor-onAssignment.png -------------------------------------------------------------------------------- /images/kafka-streams-TaskManager-addStandbyTasks.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-TaskManager-addStandbyTasks.png -------------------------------------------------------------------------------- /images/kafka-streams-TaskManager-addStreamTasks.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-TaskManager-addStreamTasks.png -------------------------------------------------------------------------------- /images/kafka-streams-TaskManager-createTasks.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-TaskManager-createTasks.png -------------------------------------------------------------------------------- /images/kafka-streams-TaskManager-hasActiveRunningTasks.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-TaskManager-hasActiveRunningTasks.png -------------------------------------------------------------------------------- /images/kafka-streams-TaskManager-hasStandbyRunningTasks.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-TaskManager-hasStandbyRunningTasks.png -------------------------------------------------------------------------------- /images/kafka-streams-TaskManager-updateNewAndRestoringTasks.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-TaskManager-updateNewAndRestoringTasks.png -------------------------------------------------------------------------------- /images/kafka-streams-TaskManager.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-TaskManager.png -------------------------------------------------------------------------------- /images/kafka-streams-Topology-InternalTopologyBuilder.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-Topology-InternalTopologyBuilder.png -------------------------------------------------------------------------------- /images/kafka-streams-WindowBytesStoreSupplier.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-WindowBytesStoreSupplier.png -------------------------------------------------------------------------------- /images/kafka-streams-WindowStoreBuilder.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/jaceklaskowski/mastering-kafka-streams-book/894cce24a9db95fc197d1fda092fa7bb048aea0a/images/kafka-streams-WindowStoreBuilder.png -------------------------------------------------------------------------------- /kafka-logging.adoc: -------------------------------------------------------------------------------- 1 | == Logging 2 | 3 | === [[log4j.properties]] Application Logging Using log4j -- `log4j.properties` Logging Configuration File 4 | 5 | .log4j.properties 6 | ``` 7 | log4j.rootLogger=INFO, stdout 8 | log4j.appender.stdout=org.apache.log4j.ConsoleAppender 9 | log4j.appender.stdout.target=System.out 10 | log4j.appender.stdout.layout=org.apache.log4j.PatternLayout 11 | log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c)%n 12 | 13 | log4j.logger.org.apache.kafka.clients.consumer.ConsumerConfig=DEBUG 14 | ``` 15 | 16 | TIP: Save `log4j.properties` in `src/main/resources` in your Kafka application's project. 17 | 18 | Kafka uses https://www.slf4j.org/index.html[Simple Logging Facade for Java (SLF4J)] for logging. 19 | 20 | [[logback-tip]] 21 | [TIP] 22 | ==== 23 | Use `slf4j-simple` library dependency in Scala applications (in `build.sbt`) for basic logging where messages of level `INFO` and higher are printed to `System.err`. 24 | 25 | .build.sbt 26 | ```scala 27 | libraryDependencies += "org.slf4j" % "slf4j-simple" % "1.8.0-alpha2" 28 | ``` 29 | 30 | Replace slf4j's simple binding to switch between logging frameworks (e.g. `slf4j-log4j12` for log4j). 31 | 32 | .build.sbt 33 | ```scala 34 | val logback = "1.2.3" 35 | libraryDependencies += "ch.qos.logback" % "logback-core" % logback 36 | libraryDependencies += "ch.qos.logback" % "logback-classic" % logback 37 | ``` 38 | ==== 39 | 40 | [NOTE] 41 | ==== 42 | 43 | With logback's configuration (as described in the <>) you may see the following messages in the logs: 44 | 45 | ``` 46 | SLF4J: Class path contains multiple SLF4J bindings. 47 | SLF4J: Found binding in [jar:file:/Users/jacek/.ivy2/cache/org.slf4j/slf4j-log4j12/jars/slf4j-log4j12-1.7.25.jar!/org/slf4j/impl/StaticLoggerBinder.class] 48 | SLF4J: Found binding in [jar:file:/Users/jacek/.ivy2/cache/ch.qos.logback/logback-classic/jars/logback-classic-1.2.3.jar!/org/slf4j/impl/StaticLoggerBinder.class] 49 | SLF4J: See http://www.slf4j.org/codes.html#multiple_bindings for an explanation. 50 | SLF4J: Actual binding is of type [org.slf4j.impl.Log4jLoggerFactory] 51 | ``` 52 | 53 | Commenting out `logback-classic` in `build.sbt` resolves it. 54 | 55 | .build.sbt 56 | ```scala 57 | val logback = "1.2.3" 58 | libraryDependencies += "ch.qos.logback" % "logback-core" % logback 59 | //libraryDependencies += "ch.qos.logback" % "logback-classic" % logback 60 | ``` 61 | 62 | FIXME: Explain why the commenting out is required? 63 | ==== 64 | -------------------------------------------------------------------------------- /kafka-streams-AbstractNotifyingBatchingRestoreCallback.adoc: -------------------------------------------------------------------------------- 1 | == [[AbstractNotifyingBatchingRestoreCallback]] AbstractNotifyingBatchingRestoreCallback 2 | 3 | `AbstractNotifyingBatchingRestoreCallback` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-AbstractNotifyingRestoreCallback.adoc: -------------------------------------------------------------------------------- 1 | == [[AbstractNotifyingRestoreCallback]] AbstractNotifyingRestoreCallback 2 | 3 | `AbstractNotifyingRestoreCallback` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-BatchingStateRestoreCallback.adoc: -------------------------------------------------------------------------------- 1 | == [[BatchingStateRestoreCallback]] BatchingStateRestoreCallback 2 | 3 | `BatchingStateRestoreCallback` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-Cancellable.adoc: -------------------------------------------------------------------------------- 1 | == [[Cancellable]] Cancellable 2 | 3 | `Cancellable` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-DefaultPartitionGrouper.adoc: -------------------------------------------------------------------------------- 1 | == [[DefaultPartitionGrouper]] DefaultPartitionGrouper 2 | 3 | `DefaultPartitionGrouper` is a concrete <> that...FIXME 4 | 5 | === [[partitionGroups]] `partitionGroups` Method 6 | 7 | [source, java] 8 | ---- 9 | Map> partitionGroups( 10 | final Map> topicGroups, 11 | final Cluster metadata) 12 | ---- 13 | 14 | NOTE: `partitionGroups` is part of the <> to...FIXME. 15 | 16 | `partitionGroups`...FIXME 17 | 18 | === [[maxNumPartitions]] Finding Maximum Number Of Partitions For Topics -- `maxNumPartitions` Method 19 | 20 | [source, java] 21 | ---- 22 | int maxNumPartitions( 23 | final Cluster metadata, 24 | final Set topics) 25 | ---- 26 | 27 | `maxNumPartitions`...FIXME 28 | 29 | NOTE: `maxNumPartitions` is used exclusively when `DefaultPartitionGrouper` is requested to <>. 30 | -------------------------------------------------------------------------------- /kafka-streams-DelegatingStateRestoreListener.adoc: -------------------------------------------------------------------------------- 1 | == [[DelegatingStateRestoreListener]] DelegatingStateRestoreListener 2 | 3 | `DelegatingStateRestoreListener` is a concrete <> that simply intercepts the state-related events (e.g. <>, <> and <>) and notifies the <>. 4 | 5 | NOTE: `DelegatingStateRestoreListener` is a final class of <> class and so has access to the internals of a `KafkaStreams` instance. 6 | 7 | `DelegatingStateRestoreListener` is <> for a <> (that uses it to create a <> and <>). 8 | 9 | [[creating-instance]] 10 | `DelegatingStateRestoreListener` takes no arguments when created. 11 | 12 | === [[onBatchRestored]] `onBatchRestored` Method 13 | 14 | [source, java] 15 | ---- 16 | void onBatchRestored( 17 | final TopicPartition topicPartition, 18 | final String storeName, 19 | final long batchEndOffset, 20 | final long numRestored) 21 | ---- 22 | 23 | NOTE: `onBatchRestored` is part of the <> to...FIXME. 24 | 25 | `onBatchRestored`...FIXME 26 | 27 | === [[onRestoreEnd]] `onRestoreEnd` Method 28 | 29 | [source, java] 30 | ---- 31 | void onRestoreEnd( 32 | final TopicPartition topicPartition, 33 | final String storeName, 34 | final long batchEndOffset, 35 | final long numRestored) 36 | ---- 37 | 38 | NOTE: `onRestoreEnd` is part of the <> to...FIXME. 39 | 40 | `onRestoreEnd`...FIXME 41 | 42 | === [[onRestoreStart]] `onRestoreStart` Method 43 | 44 | [source, java] 45 | ---- 46 | void onRestoreStart( 47 | final TopicPartition topicPartition, 48 | final String storeName, 49 | final long batchEndOffset, 50 | final long numRestored) 51 | ---- 52 | 53 | NOTE: `onRestoreStart` is part of the <> to...FIXME. 54 | 55 | `onRestoreStart`...FIXME 56 | -------------------------------------------------------------------------------- /kafka-streams-ExtractRecordMetadataTimestamp.adoc: -------------------------------------------------------------------------------- 1 | == [[ExtractRecordMetadataTimestamp]] ExtractRecordMetadataTimestamp 2 | 3 | `ExtractRecordMetadataTimestamp` is the <> of <> that use an <> when a record has a negative (invalid) timestamp value (while <> embedded metadata timestamps from Kafka messages). 4 | 5 | [[contract]] 6 | [source, java] 7 | ---- 8 | package org.apache.kafka.streams.processor; 9 | 10 | abstract class ExtractRecordMetadataTimestamp implements TimestampExtractor { 11 | // only required methods that have no implementation 12 | // the others follow 13 | abstract long onInvalidTimestamp( 14 | final ConsumerRecord record, 15 | final long recordTimestamp, 16 | final long previousTimestamp); 17 | } 18 | ---- 19 | 20 | NOTE: `ExtractRecordMetadataTimestamp` is an `Evolving` contract which means that compatibility may be broken at a minor release. 21 | 22 | .ExtractRecordMetadataTimestamp Contract 23 | [cols="1,2",options="header",width="100%"] 24 | |=== 25 | | Method 26 | | Description 27 | 28 | | `onInvalidTimestamp` 29 | | [[onInvalidTimestamp]] Used when...FIXME 30 | |=== 31 | 32 | [[implementations]] 33 | .ExtractRecordMetadataTimestamps 34 | [cols="1,2",options="header",width="100%"] 35 | |=== 36 | | ExtractRecordMetadataTimestamp 37 | | Description 38 | 39 | | link:kafka-streams-FailOnInvalidTimestamp.adoc[FailOnInvalidTimestamp] 40 | | [[FailOnInvalidTimestamp]] 41 | 42 | | `UsePreviousTimeOnInvalidTimestamp` 43 | | [[UsePreviousTimeOnInvalidTimestamp]] 44 | 45 | | `LogAndSkipOnInvalidTimestamp` 46 | | [[LogAndSkipOnInvalidTimestamp]] 47 | |=== 48 | 49 | === [[extract]] `extract` Method 50 | 51 | [source, java] 52 | ---- 53 | long extract(final ConsumerRecord record, final long previousTimestamp) 54 | ---- 55 | 56 | NOTE: `extract` is part of link:kafka-streams-TimestampExtractor.adoc#extract[TimestampExtractor Contract] to extract a timestamp from a record. 57 | 58 | `extract`...FIXME 59 | -------------------------------------------------------------------------------- /kafka-streams-FailOnInvalidTimestamp.adoc: -------------------------------------------------------------------------------- 1 | == [[FailOnInvalidTimestamp]] FailOnInvalidTimestamp Timestamp Extractor 2 | 3 | `FailOnInvalidTimestamp` is used as the <>. 4 | -------------------------------------------------------------------------------- /kafka-streams-GlobalKTable.adoc: -------------------------------------------------------------------------------- 1 | == [[GlobalKTable]] GlobalKTable -- Global Changelog Stream 2 | 3 | `GlobalKTable` is the <> of <> that are global and have a local state store registered under the optional <>. 4 | 5 | [[contract]] 6 | .GlobalKTable Contract 7 | [cols="1m,2",options="header",width="100%"] 8 | |=== 9 | | Method 10 | | Description 11 | 12 | | queryableStoreName 13 | a| [[queryableStoreName]] 14 | 15 | [source, java] 16 | ---- 17 | String queryableStoreName() 18 | ---- 19 | 20 | Name of the local state store that can be used in interactive queries. If `null`, the local state store of the `GlobalKTable` cannot be queried (and so the `GlobalKTable` is not queryable). 21 | 22 | |=== 23 | 24 | NOTE: `GlobalKTable` is an `Evolving` contract which means that compatibility may be broken at a minor release. 25 | 26 | [[implementations]] 27 | NOTE: <> is the one and only known implementation of <> in Kafka Streams. 28 | -------------------------------------------------------------------------------- /kafka-streams-JoinWindows.adoc: -------------------------------------------------------------------------------- 1 | == [[JoinWindows]] JoinWindows -- Window Specification for Streaming Joins 2 | 3 | `JoinWindows` is a <> that is used for streaming joins. 4 | -------------------------------------------------------------------------------- /kafka-streams-Joined.adoc: -------------------------------------------------------------------------------- 1 | == [[Joined]] Joined -- Metadata for Joined Streams 2 | 3 | `Joined` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-KGroupedTable.adoc: -------------------------------------------------------------------------------- 1 | == [[KGroupedTable]] KGroupedTable Contract 2 | 3 | `KGroupedTable` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-KeyValueBytesStoreSupplier.adoc: -------------------------------------------------------------------------------- 1 | == [[KeyValueBytesStoreSupplier]] KeyValueBytesStoreSupplier Contract 2 | 3 | `KeyValueBytesStoreSupplier` is the <> of the <> for <> of <> (of `Bytes` keys and `byte[]` values, i.e. `StoreSupplier>`). 4 | 5 | [[contract]] 6 | `KeyValueBytesStoreSupplier` offers no new methods and simply specifies the type of the <> to <>, i.e. `KeyValueStore`. 7 | 8 | [[implementations]] 9 | .KeyValueBytesStoreSuppliers 10 | [cols="30,70",options="header",width="100%"] 11 | |=== 12 | | KeyValueBytesStoreSupplier 13 | | Description 14 | 15 | | <> 16 | | [[inMemoryKeyValueStore]] 17 | 18 | | <> 19 | | [[lruMap]] 20 | 21 | | <> 22 | | [[RocksDbKeyValueBytesStoreSupplier]] 23 | 24 | |=== 25 | -------------------------------------------------------------------------------- /kafka-streams-KeyValueMapper.adoc: -------------------------------------------------------------------------------- 1 | == [[KeyValueMapper]] KeyValueMapper 2 | 3 | `KeyValueMapper` is the <> of *key-value mappers* that <>. 4 | 5 | `KeyValueMapper` is the input argument of the following operators: 6 | 7 | * link:kafka-streams-KStream.adoc#selectKey[KStream.selectKey] 8 | * link:kafka-streams-KStream.adoc#map[KStream.map] 9 | * link:kafka-streams-KStream.adoc#flatMap[KStream.flatMap] 10 | * link:kafka-streams-KStream.adoc#groupBy[KStream.groupBy] 11 | * link:kafka-streams-KStream.adoc#join[KStream.join] 12 | * link:kafka-streams-KStream.adoc#leftJoin[KStream.leftJoin] 13 | 14 | * link:kafka-streams-KTable.adoc#toStream[KTable.toStream] 15 | * link:kafka-streams-KTable.adoc#groupBy[KTable.groupBy] 16 | 17 | * link:kafka-streams-Printed.adoc#withKeyValueMapper[Printed.withKeyValueMapper] 18 | 19 | [[contract]] 20 | [source, java] 21 | ---- 22 | package org.apache.kafka.streams.kstream; 23 | 24 | interface KeyValueMapper { 25 | VR apply(final K key, final V value); 26 | } 27 | ---- 28 | 29 | .KeyValueMapper Contract 30 | [cols="1,2",options="header",width="100%"] 31 | |=== 32 | | Method 33 | | Description 34 | 35 | | `apply` 36 | | [[apply]] Used when...FIXME 37 | |=== 38 | -------------------------------------------------------------------------------- /kafka-streams-PartitionGrouper.adoc: -------------------------------------------------------------------------------- 1 | == [[PartitionGrouper]] PartitionGrouper Contract 2 | 3 | `PartitionGrouper` is the <> of <> that can <>. 4 | 5 | [[contract]] 6 | .PartitionGrouper Contract 7 | [cols="1m,3",options="header",width="100%"] 8 | |=== 9 | | Method 10 | | Description 11 | 12 | | partitionGroups 13 | a| [[partitionGroups]] 14 | 15 | [source, java] 16 | ---- 17 | Map> partitionGroups( 18 | Map> topicGroups, Cluster metadata) 19 | ---- 20 | 21 | Used exclusively when `StreamsPartitionAssignor` is requested to <> 22 | |=== 23 | 24 | [[implementations]] 25 | NOTE: <> is the one and only known implementation of the <>. 26 | -------------------------------------------------------------------------------- /kafka-streams-Printed.adoc: -------------------------------------------------------------------------------- 1 | == [[Printed]] Printed 2 | 3 | `Printed` is...FIXME 4 | 5 | [[operators]] 6 | .Printed Operators 7 | [cols="1,2",options="header",width="100%"] 8 | |=== 9 | | Operator 10 | | Description 11 | 12 | | <> 13 | a| 14 | 15 | [source, java] 16 | ---- 17 | Printed withKeyValueMapper(final KeyValueMapper mapper) 18 | ---- 19 | |=== 20 | 21 | === [[withKeyValueMapper]] `withKeyValueMapper` Method 22 | 23 | [source, java] 24 | ---- 25 | Printed withKeyValueMapper(final KeyValueMapper mapper) 26 | ---- 27 | 28 | `withKeyValueMapper`...FIXME 29 | 30 | NOTE: `withKeyValueMapper` is used when...FIXME 31 | -------------------------------------------------------------------------------- /kafka-streams-PunctuationSchedule.adoc: -------------------------------------------------------------------------------- 1 | == [[PunctuationSchedule]] PunctuationSchedule -- Orderable ProcessorNodes At Timestamp 2 | 3 | `PunctuationSchedule` is a link:kafka-streams-Stamped.adoc[Stamped] with <> (as link:kafka-streams-Stamped.adoc#value[values]). 4 | 5 | In other words, `PunctuationSchedule` represents a <> at a given <> and can be ordered in ascending order. 6 | 7 | `PunctuationSchedule` is <> when: 8 | 9 | * `StreamTask` is requested to link:kafka-streams-internals-StreamTask.adoc#schedule[schedule] 10 | 11 | * `PunctuationSchedule` is requested for the <> 12 | 13 | [[creating-instance]] 14 | `PunctuationSchedule` takes the following when created: 15 | 16 | * [[node]] link:kafka-streams-internals-ProcessorNode.adoc[ProcessorNode] 17 | * [[time]] Timestamp 18 | * [[interval]] Interval 19 | * [[punctuator]] link:kafka-streams-Punctuator.adoc[Punctuator] 20 | * [[cancellable]] `RepointableCancellable` 21 | 22 | === [[next]] `next` Method 23 | 24 | [source, java] 25 | ---- 26 | PunctuationSchedule next(final long currTimestamp) 27 | ---- 28 | 29 | `next`...FIXME 30 | 31 | NOTE: `next` is used exclusively when `PunctuationQueue` is requested to link:kafka-streams-internals-PunctuationQueue.adoc#mayPunctuate[mayPunctuate]. 32 | -------------------------------------------------------------------------------- /kafka-streams-Punctuator.adoc: -------------------------------------------------------------------------------- 1 | == [[Punctuator]] Punctuator Contract -- Scheduled Periodic Actions 2 | 3 | `Punctuator` is the <> that Kafka Streams developers use to <> (aka *punctuate*) on a record stream. 4 | 5 | [[contract]] 6 | [[punctuate]] 7 | [source, java] 8 | ---- 9 | void punctuate(long timestamp) 10 | ---- 11 | 12 | `punctuate` is used exclusively when `ProcessorNode` is requested to <>. 13 | 14 | `Punctuator` is scheduled (registered) using <> method. 15 | -------------------------------------------------------------------------------- /kafka-streams-ReadOnlyKeyValueStore.adoc: -------------------------------------------------------------------------------- 1 | == [[ReadOnlyKeyValueStore]] ReadOnlyKeyValueStore 2 | 3 | `ReadOnlyKeyValueStore` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-RecordContext.adoc: -------------------------------------------------------------------------------- 1 | == [[RecordContext]] RecordContext -- Record Metadata 2 | 3 | `RecordContext` is the <> of <> that Kafka Streams developers use for *dynamic routing of output records* (using <>) based on the metadata of a record, i.e. <>, <>, <>, <>, and <>. 4 | 5 | [[implementations]] 6 | NOTE: <> is the default implementation of the <> in Kafka Streams. 7 | 8 | [[contract]] 9 | .RecordContext Contract 10 | [cols="1m,3",options="header",width="100%"] 11 | |=== 12 | | Method 13 | | Description 14 | 15 | | headers 16 | a| [[headers]] 17 | 18 | [source, java] 19 | ---- 20 | Headers headers() 21 | ---- 22 | 23 | The https://kafka.apache.org/22/javadoc/org/apache/kafka/common/header/Headers.html[headers] of the record 24 | 25 | | offset 26 | a| [[offset]] 27 | 28 | [source, java] 29 | ---- 30 | long offset() 31 | ---- 32 | 33 | The position of the record in the corresponding Kafka partition 34 | 35 | | partition 36 | a| [[partition]] 37 | 38 | [source, java] 39 | ---- 40 | int partition() 41 | ---- 42 | 43 | The partition the record has been received from 44 | 45 | | timestamp 46 | a| [[timestamp]] 47 | 48 | [source, java] 49 | ---- 50 | long timestamp() 51 | ---- 52 | 53 | The timestamp of the record 54 | 55 | | topic 56 | a| [[topic]] 57 | 58 | [source, java] 59 | ---- 60 | String topic() 61 | ---- 62 | 63 | The topic the record has been received from 64 | 65 | |=== 66 | -------------------------------------------------------------------------------- /kafka-streams-SessionBytesStoreSupplier.adoc: -------------------------------------------------------------------------------- 1 | == [[SessionBytesStoreSupplier]] SessionBytesStoreSupplier Contract 2 | 3 | `SessionBytesStoreSupplier` is the <> of the <> for <> that <>. 4 | 5 | [[implementations]] 6 | .SessionBytesStoreSuppliers 7 | [cols="30,70",options="header",width="100%"] 8 | |=== 9 | | SessionBytesStoreSupplier 10 | | Description 11 | 12 | | <> 13 | | [[InMemorySessionBytesStoreSupplier]] 14 | 15 | | <> 16 | | [[RocksDbSessionBytesStoreSupplier]] 17 | 18 | |=== 19 | -------------------------------------------------------------------------------- /kafka-streams-SessionWindowedKStream.adoc: -------------------------------------------------------------------------------- 1 | == [[SessionWindowedKStream]] SessionWindowedKStream -- Session-Windowed Stream Aggregations 2 | 3 | `SessionWindowedKStream` is the <> of <> that allows Kafka Streams developers for <>, <> and <> aggregations over a session-windowed record stream. 4 | 5 | `SessionWindowedKStream` is created using <> method. 6 | 7 | [[contract]] 8 | [source, java] 9 | ---- 10 | package org.apache.kafka.streams.kstream; 11 | 12 | interface SessionWindowedKStream { 13 | KTable, VR> aggregate( 14 | final Initializer initializer, 15 | final Aggregator aggregator, 16 | final Merger sessionMerger); 17 | KTable, VR> aggregate( 18 | final Initializer initializer, 19 | final Aggregator aggregator, 20 | final Merger sessionMerger, 21 | final Materialized> materialized); 22 | KTable, Long> count(); 23 | KTable, Long> count( 24 | final Materialized> materialized); 25 | KTable, V> reduce(final Reducer reducer); 26 | KTable, V> reduce( 27 | final Reducer reducer, 28 | final Materialized> materializedAs); 29 | } 30 | ---- 31 | 32 | .SessionWindowedKStream Contract 33 | [cols="1,2",options="header",width="100%"] 34 | |=== 35 | | Method 36 | | Description 37 | 38 | | `aggregate` 39 | | [[aggregate]] 40 | 41 | | `count` 42 | | [[count]] 43 | 44 | | `reduce` 45 | | [[reduce]] 46 | |=== 47 | 48 | [[implementations]] 49 | NOTE: link:kafka-streams-internals-SessionWindowedKStreamImpl.adoc[SessionWindowedKStreamImpl] is the one and only known implementation of <> in Kafka Streams {{ book.kafka_version }}. 50 | -------------------------------------------------------------------------------- /kafka-streams-Stamped.adoc: -------------------------------------------------------------------------------- 1 | == [[Stamped]] Stamped -- Orderable Value At Timestamp 2 | 3 | `Stamped` represents a <> at a given <>. 4 | 5 | [[creating-instance]] 6 | `Stamped` takes the following when created: 7 | 8 | * [[value]] Value (of type `V`) 9 | * [[timestamp]] Timestamp 10 | 11 | [[compareTo]] 12 | `Stamped` values can be compared (and hence ordered) by <> (in ascending order). 13 | 14 | [source, scala] 15 | ---- 16 | import org.apache.kafka.streams.processor.internals.Stamped 17 | 18 | // three stampeds in ascending order 19 | val s1 = new Stamped("a", 0) 20 | val s2 = new Stamped("b", 1) 21 | val s3 = new Stamped("c", 2) 22 | 23 | // Adding the stampeds in a random order 24 | // TreeSet is a concrete SortedSet 25 | import java.util.TreeSet 26 | import collection.JavaConverters._ 27 | val stampeds = new TreeSet(Seq(s3, s1, s2).asJava) 28 | 29 | assert(stampeds.asScala == Set(s1, s2, s3)) 30 | ---- 31 | 32 | [[extensions]] 33 | .Stampeds 34 | [cols="1,2",options="header",width="100%"] 35 | |=== 36 | | Stamped 37 | | Description 38 | 39 | | link:kafka-streams-internals-StampedRecord.adoc[StampedRecord] 40 | | [[StampedRecord]] Uses Kafka `ConsumerRecords` for <> 41 | 42 | | link:kafka-streams-PunctuationSchedule.adoc[PunctuationSchedule] 43 | | [[PunctuationSchedule]] Uses link:kafka-streams-internals-ProcessorNode.adoc[ProcessorNodes] for <> 44 | |=== 45 | -------------------------------------------------------------------------------- /kafka-streams-StreamPartitioner.adoc: -------------------------------------------------------------------------------- 1 | == [[StreamPartitioner]] StreamPartitioner 2 | 3 | `StreamPartitioner` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-StreamTask-TaskMetrics.adoc: -------------------------------------------------------------------------------- 1 | == [[TaskMetrics]] TaskMetrics 2 | 3 | `TaskMetrics` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-StreamThread-StateListener.adoc: -------------------------------------------------------------------------------- 1 | == [[StateListener]] StateListener 2 | 3 | `StateListener` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-StreamsMetrics.adoc: -------------------------------------------------------------------------------- 1 | == [[StreamsMetrics]] StreamsMetrics 2 | 3 | `StreamsMetrics` is a <> of <> that <>. 4 | 5 | [[contract]] 6 | [source, java] 7 | ---- 8 | package org.apache.kafka.streams; 9 | 10 | interface StreamsMetrics { 11 | Map metrics(); 12 | Sensor addLatencyAndThroughputSensor( 13 | final String scopeName, 14 | final String entityName, 15 | final String operationName, 16 | final Sensor.RecordingLevel recordingLevel, 17 | final String... tags); 18 | void recordLatency( 19 | final Sensor sensor, 20 | final long startNs, 21 | final long endNs); 22 | Sensor addThroughputSensor( 23 | final String scopeName, 24 | final String entityName, 25 | final String operationName, 26 | final Sensor.RecordingLevel recordingLevel, 27 | final String... tags); 28 | void recordThroughput( 29 | final Sensor sensor, 30 | final long value); 31 | Sensor addSensor( 32 | final String name, 33 | final Sensor.RecordingLevel recordingLevel); 34 | Sensor addSensor( 35 | final String name, 36 | final Sensor.RecordingLevel recordingLevel, 37 | final Sensor... parents); 38 | void removeSensor(final Sensor sensor); 39 | } 40 | ---- 41 | 42 | NOTE: `StreamsMetrics` is an `Evolving` contract which means that compatibility may be broken at a minor release. 43 | 44 | .StreamsMetrics Contract 45 | [cols="1,2",options="header",width="100%"] 46 | |=== 47 | | Method 48 | | Description 49 | 50 | | `metrics` 51 | | [[metrics]] Used when...FIXME 52 | 53 | | `recordLatency` 54 | | [[recordLatency]] Recording a latency with a Kafka `Sensor` 55 | 56 | Used when...FIXME 57 | |=== 58 | 59 | [[implementations]] 60 | NOTE: link:kafka-streams-internals-StreamsMetricsImpl.adoc[StreamsMetricsImpl] is the one and only known direct implementation of <> in Kafka Streams {{ book.kafka_version }}. 61 | -------------------------------------------------------------------------------- /kafka-streams-TimeWindowedSerializer.adoc: -------------------------------------------------------------------------------- 1 | == [[TimeWindowedSerializer]] TimeWindowedSerializer 2 | 3 | `TimeWindowedSerializer` is a <> for the keys of <> and `FullTimeWindowedSerde` windowed serdes. 4 | 5 | [[creating-instance]] 6 | [[inner]] 7 | `TimeWindowedSerializer` takes a Kafka https://kafka.apache.org/22/javadoc/org/apache/kafka/common/serialization/Serializer.html[Serializer] when created. 8 | 9 | [[configure]] 10 | When requested to configure, `TimeWindowedSerializer` uses <> and <> configuration properties for the serde of keys and values, respectively, unless the <> is defined already. 11 | -------------------------------------------------------------------------------- /kafka-streams-TimestampedKeyValueStore.adoc: -------------------------------------------------------------------------------- 1 | == [[TimestampedKeyValueStore]] TimestampedKeyValueStore 2 | 3 | `TimestampedKeyValueStore` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-To.adoc: -------------------------------------------------------------------------------- 1 | == [[To]] To 2 | 3 | `To` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-TopicNameExtractor.adoc: -------------------------------------------------------------------------------- 1 | == [[TopicNameExtractor]] TopicNameExtractor -- Dynamic Routing of Output Records 2 | 3 | `TopicNameExtractor` is the <> of <> that Kafka Streams developers use for *dynamic routing of records* by <>. 4 | 5 | NOTE: `TopicNameExtractor` can be specified using the high-level <> or the low-level <> operators. 6 | 7 | [[contract]] 8 | .TopicNameExtractor Contract 9 | [cols="1m,3",options="header",width="100%"] 10 | |=== 11 | | Method 12 | | Description 13 | 14 | | extract 15 | a| [[extract]] 16 | 17 | [source, java] 18 | ---- 19 | String extract( 20 | final K key, 21 | final V value, 22 | final RecordContext recordContext) 23 | ---- 24 | 25 | Extracts the name of the topic to publish a record to 26 | 27 | Used exclusively when `SinkNode` is requested to <> 28 | 29 | |=== 30 | 31 | [[implementations]] 32 | NOTE: <> is an internal implementation of the <> in Kafka Streams. 33 | 34 | TIP: Read up on the feature in https://cwiki.apache.org/confluence/display/KAFKA/KIP-303%3A+Add+Dynamic+Routing+in+Streams+Sink[KIP-303: Add Dynamic Routing in Streams Sink]. 35 | -------------------------------------------------------------------------------- /kafka-streams-TopologyDescription-Node.adoc: -------------------------------------------------------------------------------- 1 | == [[TopologyDescription.Node]][[Node]] TopologyDescription.Node 2 | 3 | `TopologyDescription.Node` is the <> of <> that are identified by <>, and have <> and <> nodes. 4 | 5 | [[contract]] 6 | .TopologyDescription.Node Contract 7 | [cols="30m,70",options="header",width="100%"] 8 | |=== 9 | | Method 10 | | Description 11 | 12 | | name 13 | a| [[name]] 14 | 15 | [source, java] 16 | ---- 17 | String name() 18 | ---- 19 | 20 | Used when...FIXME 21 | 22 | | predecessors 23 | a| [[predecessors]] 24 | 25 | [source, java] 26 | ---- 27 | Set predecessors() 28 | ---- 29 | 30 | Used when...FIXME 31 | 32 | | successors 33 | a| [[successors]] 34 | 35 | [source, java] 36 | ---- 37 | Set successors() 38 | ---- 39 | 40 | Used when...FIXME 41 | 42 | |=== 43 | 44 | [[implementations]] 45 | .TopologyDescription.Nodes (Direct Implementations and Extensions) 46 | [cols="30,70",options="header",width="100%"] 47 | |=== 48 | | TopologyDescription.Node 49 | | Description 50 | 51 | | <> 52 | | [[AbstractNode]] 53 | 54 | | <> 55 | | [[Processor]] 56 | 57 | | <> 58 | | [[Sink]] 59 | 60 | | <> 61 | | [[Source]] 62 | 63 | |=== 64 | -------------------------------------------------------------------------------- /kafka-streams-TopologyDescription-Processor.adoc: -------------------------------------------------------------------------------- 1 | == [[TopologyDescription.Processor]][[Processor]] TopologyDescription.Processor 2 | 3 | `TopologyDescription.Processor` is the <> of the <> for <> that can have one or more connected <>. 4 | 5 | [[contract]] 6 | .TopologyDescription.Processor Contract 7 | [cols="30m,70",options="header",width="100%"] 8 | |=== 9 | | Method 10 | | Description 11 | 12 | | stores 13 | a| [[stores]] 14 | 15 | [source, java] 16 | ---- 17 | Set stores() 18 | ---- 19 | 20 | Names of the connected stores 21 | 22 | Used when...FIXME 23 | 24 | |=== 25 | 26 | [[implementations]] 27 | NOTE: <> is the default and only known implementation of the <> in Kafka Streams. 28 | -------------------------------------------------------------------------------- /kafka-streams-TopologyDescription-Sink.adoc: -------------------------------------------------------------------------------- 1 | == [[TopologyDescription.Sink]][[Sink]] TopologyDescription.Sink 2 | 3 | `TopologyDescription.Sink` is the <> of the <> for <> that can have a <> and <>. 4 | 5 | [[contract]] 6 | .TopologyDescription.Sink Contract 7 | [cols="30m,70",options="header",width="100%"] 8 | |=== 9 | | Method 10 | | Description 11 | 12 | | topic 13 | a| [[topic]] 14 | 15 | [source, java] 16 | ---- 17 | String topic() 18 | ---- 19 | 20 | Used when...FIXME 21 | 22 | | topicNameExtractor 23 | a| [[topicNameExtractor]] 24 | 25 | [source, java] 26 | ---- 27 | TopicNameExtractor topicNameExtractor() 28 | ---- 29 | 30 | Used when...FIXME 31 | 32 | |=== 33 | 34 | [[implementations]] 35 | NOTE: <> is the default and only known implementation of the <> in Kafka Streams. 36 | -------------------------------------------------------------------------------- /kafka-streams-TopologyDescription-Source.adoc: -------------------------------------------------------------------------------- 1 | == [[TopologyDescription.Source]][[Source]] TopologyDescription.Source 2 | 3 | `TopologyDescription.Source` is the <> of the <> for <> that can have <>, <>, and <>. 4 | 5 | [[contract]] 6 | .TopologyDescription.Source Contract 7 | [cols="30m,70",options="header",width="100%"] 8 | |=== 9 | | Method 10 | | Description 11 | 12 | | topicPattern 13 | a| [[topicPattern]] 14 | 15 | [source, java] 16 | ---- 17 | Pattern topicPattern() 18 | ---- 19 | 20 | Used when...FIXME 21 | 22 | | topics 23 | a| [[topics]] 24 | 25 | [source, java] 26 | ---- 27 | String topics() 28 | ---- 29 | 30 | Used when...FIXME 31 | 32 | | topicSet 33 | a| [[topicSet]] 34 | 35 | [source, java] 36 | ---- 37 | Set topicSet() 38 | ---- 39 | 40 | Used when...FIXME 41 | 42 | |=== 43 | 44 | [[implementations]] 45 | NOTE: <> is the default and only known implementation of the <> in Kafka Streams. 46 | -------------------------------------------------------------------------------- /kafka-streams-TopologyDescription.adoc: -------------------------------------------------------------------------------- 1 | == [[TopologyDescription]] TopologyDescription -- Meta Representation of Topology 2 | 3 | `TopologyDescription` is the <> of <> (_topology descriptions_) that Kafka Streams developers use to know (_describe_) the <> and <> of a <>. 4 | 5 | [[implementations]] 6 | NOTE: <> is the only available implementation of the <> in Kafka Streams. 7 | 8 | `TopologyDescription` is available using <> method. 9 | 10 | [source, scala] 11 | ---- 12 | // Scala API for Kafka Streams 13 | import org.apache.kafka.streams.scala._ 14 | import ImplicitConversions._ 15 | import Serdes._ 16 | 17 | val builder = new StreamsBuilder 18 | builder 19 | .stream[String, String]("records") 20 | .groupByKey 21 | .count 22 | .toStream 23 | .to("counts") 24 | val topology = builder.build 25 | 26 | val meta = topology.describe 27 | scala> :type meta 28 | org.apache.kafka.streams.TopologyDescription 29 | 30 | scala> println(meta.globalStores) 31 | [] 32 | 33 | scala> println(meta.subtopologies.size) 34 | 1 35 | ---- 36 | 37 | [[contract]] 38 | .TopologyDescription Contract 39 | [cols="1m,3",options="header",width="100%"] 40 | |=== 41 | | Method 42 | | Description 43 | 44 | | globalStores 45 | a| [[globalStores]] 46 | 47 | [source, java] 48 | ---- 49 | Set globalStores() 50 | ---- 51 | 52 | The global stores of a <> 53 | 54 | | subtopologies 55 | a| [[subtopologies]] 56 | 57 | [source, java] 58 | ---- 59 | Set subtopologies() 60 | ---- 61 | 62 | The subtopologies of a <> 63 | 64 | |=== 65 | -------------------------------------------------------------------------------- /kafka-streams-TopologyTestDriver.adoc: -------------------------------------------------------------------------------- 1 | == [[TopologyTestDriver]] TopologyTestDriver 2 | 3 | `TopologyTestDriver` is...FIXME 4 | 5 | === [[getAllStateStores]] `getAllStateStores` Method 6 | 7 | [source, java] 8 | ---- 9 | Map getAllStateStores() 10 | ---- 11 | 12 | `getAllStateStores`...FIXME 13 | 14 | NOTE: `getAllStateStores` is used when...FIXME 15 | 16 | === [[pipeInput]] `pipeInput` Method 17 | 18 | [source, java] 19 | ---- 20 | void pipeInput(final ConsumerRecord consumerRecord) 21 | ---- 22 | 23 | `pipeInput`...FIXME 24 | 25 | NOTE: `pipeInput` is used when...FIXME 26 | -------------------------------------------------------------------------------- /kafka-streams-TransformerSupplier.adoc: -------------------------------------------------------------------------------- 1 | == [[TransformerSupplier]] TransformerSupplier 2 | 3 | `TransformerSupplier` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-UnlimitedWindows.adoc: -------------------------------------------------------------------------------- 1 | == [[UnlimitedWindows]] UnlimitedWindows 2 | 3 | `UnlimitedWindows` is a <> of <>. 4 | -------------------------------------------------------------------------------- /kafka-streams-ValueJoiner.adoc: -------------------------------------------------------------------------------- 1 | == [[ValueJoiner]] ValueJoiner 2 | 3 | `ValueJoiner` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-ValueTransformer.adoc: -------------------------------------------------------------------------------- 1 | == [[ValueTransformer]] ValueTransformer -- Stateful Record-by-Record Value Transformation 2 | 3 | `ValueTransformer` is the <> in Kafka Streams for *stateful mapping* of a value to a new value of an arbitrary type. 4 | 5 | [[contract]] 6 | [source, java] 7 | ---- 8 | package org.apache.kafka.streams.kstream; 9 | 10 | interface ValueTransformer { 11 | void close(); 12 | void init(final ProcessorContext context); 13 | VR transform(final V value); 14 | // deprecated 15 | VR punctuate(final long timestamp); 16 | } 17 | ---- 18 | 19 | `ValueTransformer` is used when a stream (`AbstractStream`) is requested to `toInternalValueTransformerSupplier`. 20 | 21 | .ValueTransformer Contract 22 | [cols="1,2",options="header",width="100%"] 23 | |=== 24 | | Method 25 | | Description 26 | 27 | | [[close]] `close` 28 | | Used when...FIXME 29 | 30 | | [[init]] `init` 31 | | Used when...FIXME 32 | 33 | | [[transform]] `transform` 34 | | Used when...FIXME 35 | 36 | | [[punctuate]] `punctuate` 37 | | *DEPRECATED* Use link:kafka-streams-Punctuator.adoc[Punctuator] interface 38 | |=== 39 | -------------------------------------------------------------------------------- /kafka-streams-ValueTransformerSupplier.adoc: -------------------------------------------------------------------------------- 1 | == [[ValueTransformerSupplier]] ValueTransformerSupplier -- ValueTransformers Object Factory 2 | 3 | `ValueTransformerSupplier` is a <> in Kafka Streams for *object factories* (aka _suppliers_) that <>. 4 | 5 | [[contract]] 6 | [[get]] 7 | [source, java] 8 | ---- 9 | package org.apache.kafka.streams.kstream; 10 | 11 | interface ValueTransformerSupplier { 12 | ValueTransformer get(); 13 | } 14 | ---- 15 | 16 | `ValueTransformerSupplier` is used exclusively in link:kafka-streams-KStream.adoc#transformValues[KStream.transformValues] stateful transformation (of record values in a stream) to get a new link:kafka-streams-ValueTransformer.adoc[ValueTransformer] where the transformation happens. 17 | 18 | [source, scala] 19 | ---- 20 | FIXME Example of KStream.transformValues 21 | ---- 22 | 23 | `get`...FIXME 24 | -------------------------------------------------------------------------------- /kafka-streams-ValueTransformerWithKey.adoc: -------------------------------------------------------------------------------- 1 | == [[ValueTransformerWithKey]] ValueTransformerWithKey 2 | 3 | `ValueTransformerWithKey` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-WallclockTimestampExtractor.adoc: -------------------------------------------------------------------------------- 1 | == [[WallclockTimestampExtractor]] WallclockTimestampExtractor 2 | 3 | `WallclockTimestampExtractor` is a link:kafka-streams-TimestampExtractor.adoc[TimestampExtractor] that <> when requested to extract a timestamp from a record. 4 | 5 | === [[extract]] `extract` Method 6 | 7 | [source, java] 8 | ---- 9 | long extract(final ConsumerRecord record, final long previousTimestamp) 10 | ---- 11 | 12 | NOTE: `extract` is part of link:kafka-streams-TimestampExtractor.adoc#extract[TimestampExtractor Contract] to extract a timestamp from a record. 13 | 14 | `extract` simply requests the Java https://docs.oracle.com/javase/8/docs/api/java/lang/System.html[System] for the link:++https://docs.oracle.com/javase/8/docs/api/java/lang/System.html#currentTimeMillis--++[current time in milliseconds]. 15 | -------------------------------------------------------------------------------- /kafka-streams-Window.adoc: -------------------------------------------------------------------------------- 1 | == [[Window]] Window 2 | 3 | `Window` is the <> of <> that are described by the <> and <> timestamps and can be checked if they <>. 4 | 5 | [[contract]] 6 | [source, java] 7 | ---- 8 | package org.apache.kafka.streams.kstream; 9 | 10 | abstract class Window { 11 | // only required methods that have no implementation 12 | // the others follow 13 | public abstract boolean overlap(final Window other); 14 | } 15 | ---- 16 | 17 | .(Subset of) Window Contract 18 | [cols="1,2",options="header",width="100%"] 19 | |=== 20 | | Method 21 | | Description 22 | 23 | | `overlap` 24 | | [[overlap]] Checks whether a window overlaps with another. 25 | |=== 26 | 27 | [[start]] 28 | Use `start` method to access the <> of a window. 29 | 30 | [source, java] 31 | ---- 32 | long start() 33 | ---- 34 | 35 | [[end]] 36 | Use `end` method to access the <> of a window. 37 | 38 | [source, java] 39 | ---- 40 | long end() 41 | ---- 42 | 43 | `Window` has the following text representation: 44 | 45 | ``` 46 | Window{start=[startMs], end=[endMs]} 47 | ``` 48 | 49 | [[extensions]] 50 | .Windows 51 | [cols="1,2",options="header",width="100%"] 52 | |=== 53 | | Window 54 | | Description 55 | 56 | | link:kafka-streams-internals-SessionWindow.adoc[SessionWindow] 57 | | [[SessionWindow]] 58 | 59 | | `TimeWindow` 60 | | [[TimeWindow]] 61 | 62 | | `UnlimitedWindow` 63 | | [[UnlimitedWindow]] 64 | |=== 65 | 66 | === [[creating-instance]] Creating Window Instance 67 | 68 | `Window` takes the following when created: 69 | 70 | * [[startMs]] The start timestamp of the window (in milliseconds) 71 | * [[endMs]] The end timestamp of the window (in milliseconds) 72 | 73 | `Window` initializes the <>. 74 | 75 | `Window` is <> when...MEFIXME 76 | 77 | `Window` is <> along with...MEFIXME 78 | 79 | NOTE: `Window` is a Java/Scala abstract class and cannot be <> directly. It is created indirectly when the <> are. 80 | -------------------------------------------------------------------------------- /kafka-streams-WindowBytesStoreSupplier.adoc: -------------------------------------------------------------------------------- 1 | == [[WindowBytesStoreSupplier]] WindowBytesStoreSupplier Contract -- StoreSuppliers of WindowStores 2 | 3 | [[get]] 4 | `WindowBytesStoreSupplier` is the <> of the <> for <> that <> (i.e. <> of type ``). 5 | 6 | .WindowBytesStoreSupplier et al. 7 | image::images/kafka-streams-WindowBytesStoreSupplier.png[align="center"] 8 | 9 | [[implementations]] 10 | NOTE: <> is the one and only known implementation of the <>. 11 | 12 | [[contract]] 13 | .WindowBytesStoreSupplier Contract 14 | [cols="1m,2",options="header",width="100%"] 15 | |=== 16 | | Property 17 | | Description 18 | 19 | | retainDuplicates 20 | a| [[retainDuplicates]] 21 | 22 | [source, java] 23 | ---- 24 | boolean retainDuplicates() 25 | ---- 26 | 27 | Used exclusively when `WindowStoreBuilder` is requested to <>. 28 | 29 | | retentionPeriod 30 | a| [[retentionPeriod]] 31 | 32 | [source, java] 33 | ---- 34 | long retentionPeriod() 35 | ---- 36 | 37 | Used exclusively when `WindowStoreBuilder` is requested for the <>. 38 | 39 | | segmentIntervalMs 40 | a| [[segmentIntervalMs]] 41 | 42 | [source, java] 43 | ---- 44 | long segmentIntervalMs() 45 | ---- 46 | 47 | Used exclusively when `WindowStoreBuilder` is requested to <>. 48 | 49 | | windowSize 50 | a| [[windowSize]] 51 | 52 | [source, java] 53 | ---- 54 | long windowSize() 55 | ---- 56 | 57 | Used exclusively when `WindowStoreBuilder` is requested to <>. 58 | 59 | |=== 60 | -------------------------------------------------------------------------------- /kafka-streams-WindowStore.adoc: -------------------------------------------------------------------------------- 1 | == [[WindowStore]] WindowStore 2 | 3 | `WindowStore` is the <> of the <> for <> that are `ReadOnlyWindowStores` and can <> a record with an optional timestamp. 4 | 5 | [[contract]] 6 | .WindowStore Contract 7 | [cols="1m,2",options="header",width="100%"] 8 | |=== 9 | | Method 10 | | Description 11 | 12 | | put 13 | a| [[put]] 14 | 15 | [source, java] 16 | ---- 17 | void put(K key, V value) // <1> 18 | void put(K key, V value, long timestamp) 19 | ---- 20 | <1> Uses the current wall-clock time as the timestamp 21 | 22 | Used when <> and <> are requested to process a record 23 | 24 | |=== 25 | 26 | [[implementations]] 27 | .WindowStores 28 | [cols="1,2",options="header",width="100%"] 29 | |=== 30 | | WindowStore 31 | | Description 32 | 33 | | <> 34 | | [[CachingWindowStore]] 35 | 36 | | <> 37 | | [[ChangeLoggingWindowBytesStore]] 38 | 39 | | <> 40 | | [[MeteredWindowStore]] 41 | 42 | | <> 43 | | [[RocksDBWindowStore]] 44 | 45 | |=== 46 | -------------------------------------------------------------------------------- /kafka-streams-Windowed.adoc: -------------------------------------------------------------------------------- 1 | == [[Windowed]] Windowed 2 | 3 | `Windowed` is the result key type of a windowed stream aggregation. 4 | 5 | `Windowed` is <> when: 6 | 7 | * <> and <> processors are requested to process a record 8 | 9 | * `CachingSessionStore` state store is requested to <> 10 | 11 | * `MergedSortedCacheSessionStoreIterator` is requested to `deserializeCacheKey` 12 | 13 | * `MeteredSessionStore` is requested to <> and <> 14 | 15 | * `MeteredWindowedKeyValueIterator` is requested to `windowedKey` 16 | 17 | * `SessionKeySchema` is requested to...FIXME 18 | 19 | * `WindowKeySchema` is requested to...FIXME 20 | 21 | * `WindowStoreIteratorWrapper` is requested to...FIXME 22 | 23 | [[creating-instance]] 24 | `Windowed` takes the following when created: 25 | 26 | * [[key]] Key of `K` type 27 | * [[window]] <> 28 | -------------------------------------------------------------------------------- /kafka-streams-WindowedSerdes.adoc: -------------------------------------------------------------------------------- 1 | == [[WindowedSerdes]] WindowedSerdes -- SessionWindowedSerde and TimeWindowedSerde 2 | 3 | `WindowedSerdes` is a class that acts as a namespace for two static classes: 4 | 5 | * [[SessionWindowedSerde]] `SessionWindowedSerde` for session-windowed aggregations 6 | * [[TimeWindowedSerde]] `TimeWindowedSerde` for time-windowed aggregation 7 | 8 | `WindowedSerdes` also defines two <> to create instances of <> and <> for the specified inner class type. 9 | 10 | [[factory-methods]] 11 | .WindowedSerdes's Factory Methods 12 | [cols="1m,2",options="header",width="100%"] 13 | |=== 14 | | Method 15 | | Description 16 | 17 | | sessionWindowedSerdeFrom 18 | a| [[sessionWindowedSerdeFrom]] 19 | 20 | [source, java] 21 | ---- 22 | Serde> sessionWindowedSerdeFrom( 23 | final Class type) 24 | ---- 25 | 26 | | timeWindowedSerdeFrom 27 | a| [[timeWindowedSerdeFrom]] 28 | 29 | [source, java] 30 | ---- 31 | Serde> timeWindowedSerdeFrom( 32 | final Class type) 33 | ---- 34 | 35 | |=== 36 | 37 | === Scala API for Kafka Streams 38 | 39 | <> object defines the following implicit conversions to create instances of `TimeWindowedSerde` and `SessionWindowedSerde` (without an explicit inner class type): 40 | 41 | * <> for a <> instance 42 | 43 | * <> for a <> instance 44 | -------------------------------------------------------------------------------- /kafka-streams-exactly-once-support-eos.adoc: -------------------------------------------------------------------------------- 1 | == Exactly-Once Support (EOS) 2 | 3 | *Exactly-Once Support (EOS)* (aka *Exactly-Once Processing Guarantee*) is...FIXME 4 | 5 | [[eosEnabled]][[EXACTLY_ONCE]] 6 | `StreamsConfig` defines <> internal flag that is enabled (`true`) when <> is <> (`exactly_once`). 7 | 8 | <> can be one of the two values: 9 | 10 | * <> (default) 11 | 12 | * <> 13 | 14 | With EOS enabled, the frequency with which to save the position of a processor (<>) is always `100L`. 15 | 16 | Exactly-once processing requires a cluster of at least three brokers (by default) what is the recommended setting for production. For development you can change this, by adjusting broker setting `transaction.state.log.replication.factor` and `transaction.state.log.min.isr`. 17 | 18 | === Notes 19 | 20 | * <> and <> are sensitive to EOS (per <> flag) 21 | 22 | * With <> `StreamTask` uses <> when <> and requested to <> 23 | -------------------------------------------------------------------------------- /kafka-streams-internals-AbstractRocksDBSegmentedBytesStore.adoc: -------------------------------------------------------------------------------- 1 | == [[AbstractRocksDBSegmentedBytesStore]] AbstractRocksDBSegmentedBytesStore 2 | 3 | `AbstractRocksDBSegmentedBytesStore` is a concrete <> that...FIXME 4 | 5 | [[persistent]] 6 | `AbstractRocksDBSegmentedBytesStore` is always <>. 7 | -------------------------------------------------------------------------------- /kafka-streams-internals-AssignedStandbyTasks.adoc: -------------------------------------------------------------------------------- 1 | == [[AssignedStandbyTasks]] AssignedStandbyTasks -- AssignedTasks For StandbyTasks 2 | 3 | `AssignedStandbyTasks` is a concrete <> for <> that is <> exclusively for the <> for a <> (when `KafkaStreams` is <>). 4 | 5 | .AssignedStandbyTasks, TaskManager and StreamThread 6 | image::images/kafka-streams-AssignedStandbyTasks.png[align="center"] 7 | 8 | [[logContext]] 9 | [[creating-instance]] 10 | `AssignedStandbyTasks` takes a `LogContext` when created. 11 | 12 | [[taskTypeName]] 13 | `AssignedStandbyTasks` uses *standby task* for <>. 14 | 15 | [[logging]] 16 | [TIP] 17 | ==== 18 | Enable `ALL` logging level for `org.apache.kafka.streams.processor.internals.AssignedStandbyTasks` logger to see what happens inside. 19 | 20 | Add the following line to `log4j.properties`: 21 | 22 | ``` 23 | log4j.logger.org.apache.kafka.streams.processor.internals.AssignedStandbyTasks=ALL 24 | ``` 25 | 26 | Refer to <>. 27 | ==== 28 | -------------------------------------------------------------------------------- /kafka-streams-internals-BaseJoinProcessorNode.adoc: -------------------------------------------------------------------------------- 1 | == [[BaseJoinProcessorNode]] BaseJoinProcessorNode Contract 2 | 3 | `BaseJoinProcessorNode` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-BaseRepartitionNode.adoc: -------------------------------------------------------------------------------- 1 | == [[BaseRepartitionNode]] BaseRepartitionNode Contract 2 | 3 | `BaseRepartitionNode` is the base of <> that...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-CacheFlushListener.adoc: -------------------------------------------------------------------------------- 1 | == [[CacheFlushListener]] CacheFlushListener 2 | 3 | `CacheFlushListener` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-CachedStateStore.adoc: -------------------------------------------------------------------------------- 1 | == [[CachedStateStore]] CachedStateStore 2 | 3 | `CachedStateStore` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-CachingSessionStore.adoc: -------------------------------------------------------------------------------- 1 | == [[CachingSessionStore]] CachingSessionStore 2 | 3 | `CachingSessionStore` is...FIXME 4 | 5 | === [[init]] Initialize State Store -- `init` Method 6 | 7 | [source, java] 8 | ---- 9 | void init(final ProcessorContext context, final StateStore root) 10 | ---- 11 | 12 | NOTE: `init` is part of...FIXME 13 | 14 | `init`...FIXME 15 | 16 | === [[remove]] `remove` Method 17 | 18 | [source, java] 19 | ---- 20 | void remove(final Windowed sessionKey) 21 | ---- 22 | 23 | NOTE: `remove` is part of link:kafka-streams-SessionStore.adoc#remove[SessionStore Contract] to...FIXME. 24 | 25 | `remove`...FIXME 26 | 27 | === [[putAndMaybeForward]] `putAndMaybeForward` Internal Method 28 | 29 | [source, java] 30 | ---- 31 | void putAndMaybeForward(final ThreadCache.DirtyEntry entry, final InternalProcessorContext context) 32 | ---- 33 | 34 | `putAndMaybeForward`...FIXME 35 | 36 | NOTE: `putAndMaybeForward` is used when...FIXME 37 | 38 | === [[close]] Closing State Store -- `close` Method 39 | 40 | [source, java] 41 | ---- 42 | void close() 43 | ---- 44 | 45 | NOTE: `close` is part of...FIXME 46 | 47 | `close`...FIXME 48 | -------------------------------------------------------------------------------- /kafka-streams-internals-CachingWindowStore.adoc: -------------------------------------------------------------------------------- 1 | == [[CachingWindowStore]] CachingWindowStore 2 | 3 | `CachingWindowStore` is...FIXME 4 | 5 | === [[init]] Initialize State Store -- `init` Method 6 | 7 | [source, java] 8 | ---- 9 | void init(final ProcessorContext context, final StateStore root) 10 | ---- 11 | 12 | NOTE: `init` is part of...FIXME 13 | 14 | `init`...FIXME 15 | 16 | === [[remove]] `remove` Method 17 | 18 | [source, java] 19 | ---- 20 | void remove(final Windowed sessionKey) 21 | ---- 22 | 23 | NOTE: `remove` is part of link:kafka-streams-SessionStore.adoc#remove[SessionStore Contract] to...FIXME. 24 | 25 | `remove`...FIXME 26 | 27 | === [[putAndMaybeForward]] `putAndMaybeForward` Internal Method 28 | 29 | [source, java] 30 | ---- 31 | void putAndMaybeForward(final ThreadCache.DirtyEntry entry, final InternalProcessorContext context) 32 | ---- 33 | 34 | `putAndMaybeForward`...FIXME 35 | 36 | NOTE: `putAndMaybeForward` is used when...FIXME 37 | 38 | === [[close]] Closing State Store -- `close` Method 39 | 40 | [source, java] 41 | ---- 42 | void close() 43 | ---- 44 | 45 | NOTE: `close` is part of...FIXME 46 | 47 | `close`...FIXME 48 | -------------------------------------------------------------------------------- /kafka-streams-internals-ChangeLoggingSessionBytesStore.adoc: -------------------------------------------------------------------------------- 1 | == [[ChangeLoggingSessionBytesStore]] ChangeLoggingSessionBytesStore 2 | 3 | `ChangeLoggingSessionBytesStore` is a link:kafka-streams-SessionStore.adoc[SessionStore] that uses Kafka `Bytes` (immutable byte arrays) for keys. 4 | 5 | `ChangeLoggingSessionBytesStore` is <> exclusively when `SessionStoreBuilder` is requested to link:kafka-streams-internals-SessionStoreBuilder.adoc#build[build a MeteredSessionStore] (with link:kafka-streams-internals-SessionStoreBuilder.adoc#maybeWrapLogging[logging enabled]). 6 | 7 | [[creating-instance]] 8 | [[bytesStore]] 9 | `ChangeLoggingSessionBytesStore` takes a link:kafka-streams-SessionStore.adoc[SessionStore] when created. 10 | 11 | [[internal-registries]] 12 | .ChangeLoggingSessionBytesStore's Internal Properties (e.g. Registries, Counters and Flags) 13 | [cols="1,2",options="header",width="100%"] 14 | |=== 15 | | Name 16 | | Description 17 | 18 | | `changeLogger` 19 | | [[changeLogger]] link:kafka-streams-internals-StoreChangeLogger.adoc[StoreChangeLogger] 20 | 21 | Used when...FIXME 22 | 23 | | `innerStateSerde` 24 | | [[innerStateSerde]] `StateSerdes` 25 | 26 | Used when...FIXME 27 | |=== 28 | 29 | === [[init]] Initialize State Store -- `init` Method 30 | 31 | [source, java] 32 | ---- 33 | void init(final ProcessorContext context, final StateStore root) 34 | ---- 35 | 36 | NOTE: `init` is part of...FIXME 37 | 38 | `init`...FIXME 39 | 40 | === [[put]] Storing Aggregated Value for Session Key -- `put` Method 41 | 42 | [source, java] 43 | ---- 44 | void put(final Windowed sessionKey, final byte[] aggregate) 45 | ---- 46 | 47 | NOTE: `put` is part of link:kafka-streams-SessionStore.adoc#put[SessionStore Contract] to store an aggregated value for a session key. 48 | 49 | `put` requests the <> to link:kafka-streams-SessionStore.adoc#put[store the aggregated value for the given session key]. 50 | 51 | In the end, `put`...FIXME 52 | -------------------------------------------------------------------------------- /kafka-streams-internals-ChangeLoggingWindowBytesStore.adoc: -------------------------------------------------------------------------------- 1 | == [[ChangeLoggingWindowBytesStore]] ChangeLoggingWindowBytesStore 2 | 3 | `ChangeLoggingWindowBytesStore` is...FIXME 4 | 5 | === [[init]] Initialize State Store -- `init` Method 6 | 7 | [source, java] 8 | ---- 9 | void init(final ProcessorContext context, final StateStore root) 10 | ---- 11 | 12 | NOTE: `init` is part of...FIXME 13 | 14 | `init`...FIXME 15 | -------------------------------------------------------------------------------- /kafka-streams-internals-Checkpointable.adoc: -------------------------------------------------------------------------------- 1 | == [[Checkpointable]] Checkpointable Internal Contract 2 | 3 | `Checkpointable` is the internal <> for <> with associated partition offsets that can be <>. 4 | 5 | [[implementations]] 6 | NOTE: <> is the one and only known direct extension contract of the <>. 7 | 8 | [[contract]] 9 | .Checkpointable Contract 10 | [cols="1m,2",options="header",width="100%"] 11 | |=== 12 | | Method 13 | | Description 14 | 15 | | checkpoint 16 | a| [[checkpoint]] 17 | 18 | [source, java] 19 | ---- 20 | void checkpoint(final Map offsets) 21 | ---- 22 | 23 | Checkpointing offsets 24 | 25 | Used when: 26 | 27 | * `GlobalStateUpdateTask` is requested to <> 28 | 29 | * `StandbyTask` is requested to <> (when requested to <> and <>) 30 | 31 | * `StreamTask` is requested to <> (with <> disabled) and <> (with <> enabled) 32 | 33 | | checkpointed 34 | a| [[checkpointed]] 35 | 36 | [source, java] 37 | ---- 38 | Map checkpointed() 39 | ---- 40 | 41 | Used when: 42 | 43 | * `GlobalStateUpdateTask` is requested to <> 44 | 45 | * `StandbyTask` is requested to <> 46 | |=== 47 | -------------------------------------------------------------------------------- /kafka-streams-internals-ClientMetadata.adoc: -------------------------------------------------------------------------------- 1 | == [[ClientMetadata]] ClientMetadata 2 | 3 | `ClientMetadata` is...FIXME 4 | 5 | [[internal-registries]] 6 | .ClientMetadata's Internal Properties (e.g. Registries, Counters and Flags) 7 | [cols="1m,3",options="header",width="100%"] 8 | |=== 9 | | Name 10 | | Description 11 | 12 | | consumers 13 | | [[consumers]] Member IDs 14 | 15 | Used when...FIXME 16 | 17 | |=== 18 | 19 | === [[addConsumer]] `addConsumer` Method 20 | 21 | [source, java] 22 | ---- 23 | void addConsumer( 24 | final String consumerMemberId, 25 | final SubscriptionInfo info) 26 | ---- 27 | 28 | `addConsumer`...FIXME 29 | 30 | NOTE: `addConsumer` is used when...FIXME 31 | -------------------------------------------------------------------------------- /kafka-streams-internals-CompositeRestoreListener.adoc: -------------------------------------------------------------------------------- 1 | == [[CompositeRestoreListener]] CompositeRestoreListener 2 | 3 | `CompositeRestoreListener` is a concrete `BatchingStateRestoreCallback` and a <>. 4 | 5 | `CompositeRestoreListener` is <> exclusively when `ProcessorStateManager` is requested to <> (and restores it from the changelog topic). 6 | 7 | [[userRestoreListener]] 8 | `CompositeRestoreListener` uses a <> in <>, <> and <> callbacks. A `StateRestoreListener` can be assigned using <> method. 9 | 10 | [[stateRestoreCallback]] 11 | [[creating-instance]] 12 | `CompositeRestoreListener` takes a single <> when created. 13 | 14 | === [[setUserRestoreListener]] Setting StateRestoreListener -- `setUserRestoreListener` Method 15 | 16 | [source, java] 17 | ---- 18 | void setUserRestoreListener(final StateRestoreListener userRestoreListener) 19 | ---- 20 | 21 | `setUserRestoreListener` simply sets the <> to be the given <>. 22 | 23 | NOTE: `setUserRestoreListener` is used exclusively when `StateRestorer` is requested to <>. 24 | -------------------------------------------------------------------------------- /kafka-streams-internals-ConsumedInternal.adoc: -------------------------------------------------------------------------------- 1 | == [[ConsumedInternal]] ConsumedInternal -- Internal Accessors to Consumed Metadata 2 | 3 | `ConsumedInternal` is a <> that is used internally as an accessor object to the following properties of the managed <> instance: 4 | 5 | * [[keyDeserializer]] `Deserializer` of keys (only when <> for keys is defined) 6 | 7 | * [[valueDeserializer]] `Deserializer` of values (only when <> for values is defined) 8 | 9 | * [[offsetResetPolicy]] *Offset reset policy* (<>) 10 | 11 | * <> 12 | 13 | [[creating-instance]] 14 | [[consumed]] 15 | `ConsumedInternal` takes a single <>, its parts or none at all (that are used to create a `Consumed`) when created: 16 | 17 | * [[keySerde]] Key `Serde` 18 | * [[valSerde]][[valueSerde]] Value `Serde` 19 | * [[timestampExtractor]] <> 20 | * [[offsetReset]] *Offset reset policy* (<>) 21 | 22 | `ConsumedInternal` is <> when: 23 | 24 | * `StreamsBuilder` is requested to <>, <>, <>, and <> 25 | 26 | * `KStreamImpl` is requested to <> 27 | -------------------------------------------------------------------------------- /kafka-streams-internals-CopartitionedTopicsValidator.adoc: -------------------------------------------------------------------------------- 1 | == [[CopartitionedTopicsValidator]] CopartitionedTopicsValidator 2 | 3 | `CopartitionedTopicsValidator` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-DefaultKafkaClientSupplier.adoc: -------------------------------------------------------------------------------- 1 | == [[DefaultKafkaClientSupplier]] DefaultKafkaClientSupplier 2 | 3 | `DefaultKafkaClientSupplier` is the default link:kafka-streams-KafkaClientSupplier.adoc[KafkaClientSupplier] that...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-ForwardingDisabledProcessorContext.adoc: -------------------------------------------------------------------------------- 1 | == [[ForwardingDisabledProcessorContext]] ForwardingDisabledProcessorContext 2 | 3 | `ForwardingDisabledProcessorContext` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-GlobalKTableImpl.adoc: -------------------------------------------------------------------------------- 1 | == [[GlobalKTableImpl]] GlobalKTableImpl 2 | 3 | `GlobalKTableImpl` is a <> (of `K` primary keys and `V` value changes) that uses a <> for the <> when <>. 4 | 5 | `GlobalKTableImpl` is <> exclusively when `InternalStreamsBuilder` is requested to <> (exclusively when `StreamsBuilder` is requested to <>). 6 | 7 | [source, scala] 8 | ---- 9 | import org.apache.kafka.streams.StreamsBuilder 10 | val builder = new StreamsBuilder 11 | 12 | val globalTable = builder.globalTable("topic") 13 | 14 | import org.apache.kafka.streams.kstream.internals.GlobalKTableImpl 15 | val impl = globalTable.asInstanceOf[GlobalKTableImpl[_, _]] 16 | 17 | scala> println(impl.queryableStoreName) 18 | null 19 | ---- 20 | 21 | === [[creating-instance]] Creating GlobalKTableImpl Instance 22 | 23 | `GlobalKTableImpl` takes the following when created: 24 | 25 | * [[valueGetterSupplier]] link:kafka-streams-internals-KTableValueGetterSupplier.adoc[KTableValueGetterSupplier] (of `K` primary keys and `V` value changes) 26 | * [[queryable]] `queryable` flag 27 | 28 | === [[queryableStoreName]] `queryableStoreName` Method 29 | 30 | [source, java] 31 | ---- 32 | String queryableStoreName() 33 | ---- 34 | 35 | NOTE: `queryableStoreName` is part of link:kafka-streams-GlobalKTable.adoc#queryableStoreName[GlobalKTable Contract] to...FIXME. 36 | 37 | Only when <> is enabled, `queryableStoreName` requests the <> for link:kafka-streams-internals-KTableValueGetterSupplier.adoc#storeNames[storeNames] and takes the very first one. 38 | 39 | Otherwise, `queryableStoreName` returns `null`. 40 | -------------------------------------------------------------------------------- /kafka-streams-internals-GlobalProcessorContextImpl.adoc: -------------------------------------------------------------------------------- 1 | == [[GlobalProcessorContextImpl]] GlobalProcessorContextImpl 2 | 3 | `GlobalProcessorContextImpl` is a concrete <> that...FIXME 4 | 5 | === [[getStateStore]] `getStateStore` Method 6 | 7 | [source, java] 8 | ---- 9 | StateStore getStateStore(final String name) 10 | ---- 11 | 12 | NOTE: `getStateStore` is part of the <> to...FIXME. 13 | 14 | `getStateStore` simply requests the <> to <> by the given name. 15 | 16 | === [[forward]] `forward` Method 17 | 18 | [source, java] 19 | ---- 20 | void forward(final K key, final V value) 21 | ---- 22 | 23 | NOTE: `forward` is part of the <> to...FIXME. 24 | 25 | `forward`...FIXME 26 | -------------------------------------------------------------------------------- /kafka-streams-internals-GlobalStateMaintainer.adoc: -------------------------------------------------------------------------------- 1 | == [[GlobalStateMaintainer]] GlobalStateMaintainer 2 | 3 | `GlobalStateMaintainer` is the <> for managing global state stores. 4 | 5 | [source, java] 6 | ---- 7 | package org.apache.kafka.streams.processor.internals; 8 | 9 | interface GlobalStateMaintainer { 10 | void close() throws IOException; 11 | void flushState(); 12 | Map initialize(); 13 | void update(ConsumerRecord record); 14 | } 15 | ---- 16 | 17 | .GlobalStateMaintainer Contract 18 | [cols="1m,2",options="header",width="100%"] 19 | |=== 20 | | Method 21 | | Description 22 | 23 | | close 24 | a| [[close]] 25 | 26 | [source, java] 27 | ---- 28 | void close() throws IOException 29 | ---- 30 | 31 | Used when...FIXME 32 | 33 | | flushState 34 | | [[flushState]] Used when...FIXME 35 | 36 | | initialize 37 | | [[initialize]] Used when...FIXME 38 | 39 | | update 40 | | [[update]] Used when...FIXME 41 | |=== 42 | 43 | NOTE: link:kafka-streams-internals-GlobalStateUpdateTask.adoc[GlobalStateUpdateTask] is the one and only `GlobalStateMaintainer`. 44 | -------------------------------------------------------------------------------- /kafka-streams-internals-GlobalStateManager.adoc: -------------------------------------------------------------------------------- 1 | == [[GlobalStateManager]] GlobalStateManager 2 | 3 | `GlobalStateManager` is the <> for custom link:kafka-streams-internals-StateManager.adoc[StateManagers] that are link:kafka-streams-internals-Checkpointable.adoc[checkpointable] and...FIXME 4 | 5 | [[contract]] 6 | [source, java] 7 | ---- 8 | package org.apache.kafka.streams.processor.internals; 9 | 10 | interface GlobalStateManager extends StateManager { 11 | Set initialize(); 12 | void setGlobalProcessorContext(final InternalProcessorContext processorContext); 13 | } 14 | ---- 15 | 16 | .GlobalStateManager Contract 17 | [cols="1,2",options="header",width="100%"] 18 | |=== 19 | | Method 20 | | Description 21 | 22 | | [[initialize]] `initialize` 23 | | Used when...FIXME 24 | 25 | | [[setGlobalProcessorContext]] `setGlobalProcessorContext` 26 | | Used when...FIXME 27 | |=== 28 | -------------------------------------------------------------------------------- /kafka-streams-internals-GlobalStateStoreProvider.adoc: -------------------------------------------------------------------------------- 1 | == [[GlobalStateStoreProvider]] GlobalStateStoreProvider 2 | 3 | `GlobalStateStoreProvider` is a link:kafka-streams-internals-StateStoreProvider.adoc[StateStoreProvider] that...FIXME 4 | 5 | `GlobalStateStoreProvider` is <> along with link:kafka-streams-KafkaStreams.adoc#creating-instance[KafkaStreams] (when creating the link:kafka-streams-KafkaStreams.adoc#queryableStoreProvider[QueryableStoreProvider]). 6 | 7 | [[creating-instance]] 8 | [[globalStateStores]] 9 | `GlobalStateStoreProvider` takes a collection of link:kafka-streams-StateStore.adoc[StateStores] by their names when created. 10 | 11 | === [[stores]] `stores` Method 12 | 13 | [source, java] 14 | ---- 15 | List stores( 16 | final String storeName, 17 | final QueryableStoreType queryableStoreType) 18 | ---- 19 | 20 | NOTE: `stores` is part of <> to...FIXME. 21 | 22 | `stores`...FIXME 23 | -------------------------------------------------------------------------------- /kafka-streams-internals-GlobalStoreNode.adoc: -------------------------------------------------------------------------------- 1 | == [[GlobalStoreNode]] GlobalStoreNode 2 | 3 | `GlobalStoreNode` is...FIXME 4 | 5 | === [[writeToTopology]] `writeToTopology` Method 6 | 7 | [source, java] 8 | ---- 9 | void writeToTopology(final InternalTopologyBuilder topologyBuilder) 10 | ---- 11 | 12 | NOTE: `writeToTopology` is part of the <> to...FIXME. 13 | 14 | `writeToTopology` requests the <> to <>. 15 | 16 | `writeToTopology` then requests the given `InternalTopologyBuilder` to <>. 17 | -------------------------------------------------------------------------------- /kafka-streams-internals-GlobalStreamThread.adoc: -------------------------------------------------------------------------------- 1 | == [[GlobalStreamThread]] GlobalStreamThread 2 | 3 | `GlobalStreamThread` is...FIXME 4 | 5 | `GlobalStreamThread` is <> exclusively when `KafkaStreams` is <>. 6 | 7 | [[internal-registries]] 8 | .GlobalStreamThread's Internal Properties (e.g. Registries, Counters and Flags) 9 | [cols="1m,2",options="header",width="100%"] 10 | |=== 11 | | Name 12 | | Description 13 | 14 | | cache 15 | | [[cache]] <> 16 | |=== 17 | 18 | === [[initialize]] `initialize` Internal Method 19 | 20 | [source, java] 21 | ---- 22 | StateConsumer initialize() 23 | ---- 24 | 25 | `initialize`...FIXME 26 | 27 | NOTE: `initialize` is used exclusively when `GlobalStreamThread` is <>. 28 | 29 | === [[run]] Starting Thread -- `run` Method 30 | 31 | [source, java] 32 | ---- 33 | void run() 34 | ---- 35 | 36 | NOTE: `run` is part of Java's https://docs.oracle.com/javase/9/docs/api/java/lang/Thread.html#run--[Thread Contract] to be executed by a JVM thread. 37 | 38 | `run`...FIXME 39 | 40 | NOTE: `run` is used exclusively when `KafkaStreams` is <>. 41 | 42 | === [[creating-instance]] Creating GlobalStreamThread Instance 43 | 44 | `GlobalStreamThread` takes the following when created: 45 | 46 | * [[topology]] <> 47 | * [[config]] link:kafka-streams-StreamsConfig.adoc[StreamsConfig] 48 | * [[globalConsumer]] Kafka https://kafka.apache.org/22/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html[Consumer] (`Consumer`) 49 | * [[stateDirectory]] link:kafka-streams-internals-StateDirectory.adoc[StateDirectory] 50 | * [[cacheSizeBytes]] `cacheSizeBytes` 51 | * [[metrics]] `Metrics` 52 | * [[time]] `Time` 53 | * [[threadClientId]] `threadClientId` 54 | * [[stateRestoreListener]] <> 55 | 56 | `GlobalStreamThread` initializes the <>. 57 | -------------------------------------------------------------------------------- /kafka-streams-internals-GroupedTableOperationRepartitionNodeBuilder.adoc: -------------------------------------------------------------------------------- 1 | == [[GroupedTableOperationRepartitionNodeBuilder]] GroupedTableOperationRepartitionNodeBuilder 2 | 3 | `GroupedTableOperationRepartitionNodeBuilder` is...FIXME 4 | 5 | `GroupedTableOperationRepartitionNodeBuilder` is <> when...FIXME 6 | 7 | [[creating-instance]] 8 | `GroupedTableOperationRepartitionNodeBuilder` takes no arguments when created. 9 | 10 | NOTE: `GroupedTableOperationRepartitionNodeBuilder` is a Java static inner class of <> and can only be created using the <> factory method. 11 | 12 | === [[build]] Creating GroupedTableOperationRepartitionNode Instance -- `build` Method 13 | 14 | [source, java] 15 | ---- 16 | GroupedTableOperationRepartitionNode build() 17 | ---- 18 | 19 | `build` simply creates a <>. 20 | 21 | NOTE: `build` is used exclusively when `KGroupedTableImpl` is requested to <> (for all supported streaming operators, i.e. <>, <> and <>). 22 | -------------------------------------------------------------------------------- /kafka-streams-internals-InMemoryKeyValueStore.adoc: -------------------------------------------------------------------------------- 1 | == [[InMemoryKeyValueStore]] InMemoryKeyValueStore 2 | 3 | `InMemoryKeyValueStore` is...FIXME 4 | 5 | === [[init]] Initialize State Store -- `init` Method 6 | 7 | [source, java] 8 | ---- 9 | void init(final ProcessorContext context, final StateStore root) 10 | ---- 11 | 12 | NOTE: `init` is part of <> to initialize the <>. 13 | 14 | `init`...FIXME 15 | -------------------------------------------------------------------------------- /kafka-streams-internals-InMemorySessionBytesStoreSupplier.adoc: -------------------------------------------------------------------------------- 1 | == [[InMemorySessionBytesStoreSupplier]] InMemorySessionBytesStoreSupplier 2 | 3 | `InMemorySessionBytesStoreSupplier` is a concrete <> of <> (`SessionStore`). 4 | 5 | `InMemorySessionBytesStoreSupplier` is <> when `Stores` factory is requested to <>. 6 | 7 | === [[get]] Creating State Store -- `get` Method 8 | 9 | [source, java] 10 | ---- 11 | SessionStore get() 12 | ---- 13 | 14 | NOTE: `get` is part of the <> to get a <>. 15 | 16 | `get` simply creates a new <>. 17 | -------------------------------------------------------------------------------- /kafka-streams-internals-InMemorySessionStore.adoc: -------------------------------------------------------------------------------- 1 | == [[InMemorySessionStore]] InMemorySessionStore 2 | 3 | `InMemorySessionStore` is...FIXME 4 | 5 | === [[init]] Initialize State Store -- `init` Method 6 | 7 | [source, java] 8 | ---- 9 | void init( 10 | ProcessorContext context, 11 | StateStore root) 12 | ---- 13 | 14 | NOTE: `init` is part of...FIXME 15 | 16 | `init`...FIXME 17 | -------------------------------------------------------------------------------- /kafka-streams-internals-InMemoryTimeOrderedKeyValueBuffer.adoc: -------------------------------------------------------------------------------- 1 | == [[InMemoryTimeOrderedKeyValueBuffer]] InMemoryTimeOrderedKeyValueBuffer 2 | 3 | `InMemoryTimeOrderedKeyValueBuffer` is a concrete <> that is <> exclusively when `KTableImpl` is requested to <>. 4 | 5 | === [[creating-instance]] Creating InMemoryTimeOrderedKeyValueBuffer Instance 6 | 7 | `InMemoryTimeOrderedKeyValueBuffer` takes the following to be created: 8 | 9 | * [[storeName]] Name of a state store 10 | * [[loggingEnabled]] `loggingEnabled` flag 11 | * [[keySerde]] `Serde` for keys 12 | * [[valueSerde]] `Serde` for values 13 | 14 | `InMemoryTimeOrderedKeyValueBuffer` initializes the <>. 15 | 16 | === [[build]][[Builder]][[Builder.build]] `build` Method 17 | 18 | [source, java] 19 | ---- 20 | InMemoryTimeOrderedKeyValueBuffer build() 21 | ---- 22 | 23 | NOTE: `build` is part of the <> to create (_build_) a <>. 24 | 25 | `build` simply creates a new <>. 26 | -------------------------------------------------------------------------------- /kafka-streams-internals-InMemoryWindowStore.adoc: -------------------------------------------------------------------------------- 1 | == [[InMemoryWindowStore]] InMemoryWindowStore 2 | 3 | `InMemoryWindowStore` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-InternalTopicConfig.adoc: -------------------------------------------------------------------------------- 1 | == [[InternalTopicConfig]] InternalTopicConfig 2 | 3 | `InternalTopicConfig` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-InternalTopologyBuilder-GlobalStore.adoc: -------------------------------------------------------------------------------- 1 | == [[GlobalStore]] GlobalStore 2 | 3 | `GlobalStore` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-InternalTopologyBuilder-Processor.adoc: -------------------------------------------------------------------------------- 1 | == [[InternalTopologyBuilder.Processor]][[Processor]] InternalTopologyBuilder.Processor 2 | 3 | `InternalTopologyBuilder.Processor` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-InternalTopologyBuilder-ProcessorNodeFactory.adoc: -------------------------------------------------------------------------------- 1 | == [[ProcessorNodeFactory]] ProcessorNodeFactory 2 | 3 | `ProcessorNodeFactory` is a concrete link:kafka-streams-internals-InternalTopologyBuilder-NodeFactory.adoc[NodeFactory] that can <> for a <> and optional <>. 4 | 5 | NOTE: `ProcessorNodeFactory` is a private class. 6 | 7 | `ProcessorNodeFactory` is <> exclusively when `InternalTopologyBuilder` is requested to link:kafka-streams-internals-InternalTopologyBuilder.adoc#addGlobalStore[register a global state store] or link:kafka-streams-internals-InternalTopologyBuilder.adoc#addProcessor[addProcessor]. 8 | 9 | [[stateStoreNames]] 10 | `ProcessorNodeFactory` manages *names of the state stores* that a `Processor` can be associated with (as part of a <>). 11 | 12 | [[addStateStore]] 13 | `ProcessorNodeFactory` can have state stores registered (by the name) (which happens when `InternalTopologyBuilder` is requested to link:kafka-streams-internals-InternalTopologyBuilder.adoc#addGlobalStore[register a global state store] or link:kafka-streams-internals-InternalTopologyBuilder.adoc#connectProcessorAndStateStore[connectProcessorAndStateStore]). 14 | 15 | === [[build]] `build` Factory Method 16 | 17 | [source, java] 18 | ---- 19 | ProcessorNode build() 20 | ---- 21 | 22 | NOTE: `build` is part of link:kafka-streams-internals-InternalTopologyBuilder-NodeFactory.adoc#build[NodeFactory Contract] to...FIXME. 23 | 24 | `build` requests <> to link:kafka-streams-ProcessorSupplier.adoc#get[get a Processor] and then creates a link:kafka-streams-internals-ProcessorNode.adoc#creating-instance[ProcessorNode] with the <>, the processor and the <>. 25 | 26 | === [[creating-instance]] Creating ProcessorNodeFactory Instance 27 | 28 | `ProcessorNodeFactory` takes the following when created: 29 | 30 | * [[name]] Processor name 31 | * [[predecessors]] Predecessor nodes (by name) 32 | * [[supplier]] link:kafka-streams-ProcessorSupplier.adoc[ProcessorSupplier] 33 | -------------------------------------------------------------------------------- /kafka-streams-internals-InternalTopologyBuilder-Sink.adoc: -------------------------------------------------------------------------------- 1 | == [[InternalTopologyBuilder.Sink]][[Sink]] InternalTopologyBuilder.Sink 2 | 3 | `InternalTopologyBuilder.Sink` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-InternalTopologyBuilder-SinkNodeFactory.adoc: -------------------------------------------------------------------------------- 1 | == [[SinkNodeFactory]] SinkNodeFactory 2 | 3 | `SinkNodeFactory` is a custom <> that can <> and <> a <>. 4 | 5 | `SinkNodeFactory` is <> exclusively when `InternalTopologyBuilder` is requested to <>. 6 | 7 | === [[build]] Building SinkNode -- `build` Method 8 | 9 | [source, java] 10 | ---- 11 | ProcessorNode build() 12 | ---- 13 | 14 | NOTE: `build` is part of link:kafka-streams-internals-InternalTopologyBuilder-NodeFactory.adoc#build[NodeFactory Contract] to build a <>. 15 | 16 | `build` simply creates a link:kafka-streams-internals-SinkNode.adoc#creating-instance[SinkNode] (with the link:kafka-streams-internals-InternalTopologyBuilder-NodeFactory.adoc#name[name], the <>, the <> and the <>). 17 | 18 | NOTE: The link:kafka-streams-internals-SinkNode.adoc#name[name of the topic] of the `SinkNode` is the <> with link:kafka-streams-internals-InternalTopologyBuilder.adoc#decorateTopic[application id] for an link:kafka-streams-internals-InternalTopologyBuilder.adoc#internalTopicNames[internal topic]. 19 | 20 | === [[creating-instance]] Creating SinkNodeFactory Instance 21 | 22 | `SinkNodeFactory` takes the following when created: 23 | 24 | * [[name]] Node name 25 | * [[predecessors]] Names of the predecessors 26 | * [[topic]] Topic name 27 | * [[keySerializer]] Key `Serializer` 28 | * [[valSerializer]] Value `Serializer` 29 | * [[partitioner]] link:kafka-streams-StreamPartitioner.adoc[StreamPartitioner] 30 | 31 | === [[describe]] Describing SinkNode -- `describe` Method 32 | 33 | [source, java] 34 | ---- 35 | Sink describe() 36 | ---- 37 | 38 | NOTE: `describe` is part of link:kafka-streams-internals-InternalTopologyBuilder-NodeFactory.adoc#describe[NodeFactory Contract] to describe a <>. 39 | 40 | `describe`...FIXME 41 | -------------------------------------------------------------------------------- /kafka-streams-internals-InternalTopologyBuilder-Source.adoc: -------------------------------------------------------------------------------- 1 | == [[InternalTopologyBuilder.Source]][[Source]] InternalTopologyBuilder.Source 2 | 3 | `InternalTopologyBuilder.Source` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-InternalTopologyBuilder-StateStoreFactory.adoc: -------------------------------------------------------------------------------- 1 | == [[StateStoreFactory]] StateStoreFactory 2 | 3 | `StateStoreFactory` is the internal class of <> for...FIXME 4 | 5 | `StateStoreFactory` is <> exclusively when `InternalTopologyBuilder` is requested to <>. 6 | 7 | [[creating-instance]][[builder]] 8 | `StateStoreFactory` takes a single <> to be created. 9 | -------------------------------------------------------------------------------- /kafka-streams-internals-InternalTopologyBuilder-TopologyDescription.adoc: -------------------------------------------------------------------------------- 1 | == [[InternalTopologyBuilder.TopologyDescription]] InternalTopologyBuilder.TopologyDescription 2 | 3 | `InternalTopologyBuilder.TopologyDescription` is...FIXME 4 | 5 | NOTE: Don't get confused with <> that is the contract of `InternalTopologyBuilder.TopologyDescription`. 6 | -------------------------------------------------------------------------------- /kafka-streams-internals-KStreamAggProcessorSupplier.adoc: -------------------------------------------------------------------------------- 1 | == [[KStreamAggProcessorSupplier]] KStreamAggProcessorSupplier Contract 2 | 3 | `KStreamAggProcessorSupplier` is the <> of link:kafka-streams-ProcessorSupplier.adoc[ProcessorSuppliers] that allows <> for <> and to <>. 4 | 5 | [[contract]] 6 | [source, java] 7 | ---- 8 | package org.apache.kafka.streams.kstream.internals; 9 | 10 | interface KStreamAggProcessorSupplier extends ProcessorSupplier { 11 | void enableSendingOldValues(); 12 | KTableValueGetterSupplier view(); 13 | } 14 | ---- 15 | 16 | .KStreamAggProcessorSupplier Contract 17 | [cols="1,2",options="header",width="100%"] 18 | |=== 19 | | Method 20 | | Description 21 | 22 | | `enableSendingOldValues` 23 | | [[enableSendingOldValues]] Used when...FIXME 24 | 25 | | `view` 26 | | [[view]] Used when...FIXME 27 | |=== 28 | 29 | [[implementations]] 30 | .KStreamAggProcessorSuppliers 31 | [cols="1,2",options="header",width="100%"] 32 | |=== 33 | | KStreamAggProcessorSupplier 34 | | Description 35 | 36 | | `KStreamWindowReduce` 37 | | [[KStreamWindowReduce]] 38 | 39 | | `KStreamAggregate` 40 | | [[KStreamAggregate]] 41 | 42 | | <> 43 | | [[KStreamSessionWindowAggregate]] 44 | 45 | | `KStreamReduce` 46 | | [[KStreamReduce]] 47 | 48 | | <> 49 | | [[KStreamWindowAggregate]] 50 | |=== 51 | -------------------------------------------------------------------------------- /kafka-streams-internals-KStreamAggregate.adoc: -------------------------------------------------------------------------------- 1 | == [[KStreamAggregate]] KStreamAggregate 2 | 3 | `KStreamAggregate` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-KStreamFilter.adoc: -------------------------------------------------------------------------------- 1 | == [[KStreamFilter]] KStreamFilter -- ProcessorSupplier of KStreamFilterProcessors for filter and filterNot Streaming Operators 2 | 3 | `KStreamFilter` is a <> of <>. 4 | 5 | `KStreamFilter` is <> when `KStreamImpl` is requested to execute the following streaming operators: 6 | 7 | * <> 8 | * <> 9 | * <> 10 | 11 | [[creating-instance]] 12 | `KStreamFilter` takes the following when created: 13 | 14 | * [[predicate]] `Predicate` (of `K` keys and `V` values) 15 | * [[filterNot]] `filterNot` flag 16 | 17 | [[get]] 18 | `KStreamFilter` creates a new <> whenever requested to <>. 19 | 20 | [[KStreamFilterProcessor]] 21 | `KStreamFilterProcessor` is an <> that requests the <> to <> (to downstream processors) only when the <> is met. 22 | -------------------------------------------------------------------------------- /kafka-streams-internals-KStreamFilterProcessor.adoc: -------------------------------------------------------------------------------- 1 | == [[KStreamFilterProcessor]] KStreamFilterProcessor 2 | 3 | `KStreamFilterProcessor` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-KStreamImpl-KStreamImplJoin.adoc: -------------------------------------------------------------------------------- 1 | == [[KStreamImplJoin]] KStreamImplJoin 2 | 3 | `KStreamImplJoin` is...FIXME 4 | 5 | === [[join]] `join` Method 6 | 7 | [source, java] 8 | ---- 9 | KStream join( 10 | KStream lhs, 11 | KStream other, 12 | ValueJoiner joiner, 13 | JoinWindows windows, 14 | Joined joined) 15 | ---- 16 | 17 | `join`...FIXME 18 | 19 | NOTE: `join` is used exclusively when `KStreamImpl` is requested to <> (for <>, <> and <> operators). 20 | -------------------------------------------------------------------------------- /kafka-streams-internals-KStreamJoinWindowProcessor.adoc: -------------------------------------------------------------------------------- 1 | == [[KStreamJoinWindowProcessor]] KStreamJoinWindowProcessor 2 | 3 | `KStreamJoinWindowProcessor` is a <> that is used for 4 | 5 | ??? <> stateful operator. 6 | 7 | `KStreamJoinWindowProcessor` is <> when...FIXME 8 | 9 | [[creating-instance]] 10 | `KStreamJoinWindowProcessor` takes no arguments when created. 11 | 12 | === [[init]] Initializing Processor -- `init` Method 13 | 14 | [source, java] 15 | ---- 16 | void init(ProcessorContext context) 17 | ---- 18 | 19 | NOTE: `init` is part of the <> to initialize the <>. 20 | 21 | `init`...FIXME 22 | 23 | === [[process]] Processing Record -- `process` Method 24 | 25 | [source, java] 26 | ---- 27 | void process(K key, V value) 28 | ---- 29 | 30 | NOTE: `process` is part of the <> to process a single record. 31 | 32 | `process`...FIXME 33 | -------------------------------------------------------------------------------- /kafka-streams-internals-KStreamPassThrough.adoc: -------------------------------------------------------------------------------- 1 | == [[KStreamPassThrough]] KStreamPassThrough 2 | 3 | `KStreamPassThrough` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-KStreamSessionWindowAggregate.adoc: -------------------------------------------------------------------------------- 1 | == [[KStreamSessionWindowAggregate]] KStreamSessionWindowAggregate -- ProcessorSupplier of KStreamSessionWindowAggregateProcessors 2 | 3 | `KStreamSessionWindowAggregate` is a link:kafka-streams-internals-KStreamAggProcessorSupplier.adoc[KStreamAggProcessorSupplier] that...FIXME 4 | 5 | `KStreamSessionWindowAggregate` is <> when: 6 | 7 | * `KGroupedStreamImpl` is requested to link:kafka-streams-internals-KGroupedStreamImpl.adoc#aggregate[aggregate] 8 | 9 | * `SessionWindowedKStreamImpl` is requested to link:kafka-streams-internals-SessionWindowedKStreamImpl.adoc#aggregate[aggregate] and link:kafka-streams-internals-SessionWindowedKStreamImpl.adoc#doAggregate[doAggregate] 10 | 11 | [[sendOldValues]] 12 | `KStreamSessionWindowAggregate` uses `sendOldValues` flag that is disabled (`false`) by default and can be <>. 13 | 14 | === [[get]] `get` Method 15 | 16 | [source, java] 17 | ---- 18 | Processor get() 19 | ---- 20 | 21 | NOTE: `get` is part of link:kafka-streams-ProcessorSupplier.adoc#get[ProcessorSupplier Contract] to supply a stream processor. 22 | 23 | `get`...FIXME 24 | 25 | === [[creating-instance]] Creating KStreamSessionWindowAggregate Instance 26 | 27 | `KStreamSessionWindowAggregate` takes the following when created: 28 | 29 | * [[windows]] `SessionWindows` 30 | * [[storeName]] Store name 31 | * [[initializer]] `Initializer` 32 | * [[aggregator]] `Aggregator` 33 | * [[sessionMerger]] `Merger` 34 | 35 | === [[enableSendingOldValues]] `enableSendingOldValues` Method 36 | 37 | [source, java] 38 | ---- 39 | void enableSendingOldValues() 40 | ---- 41 | 42 | NOTE: `enableSendingOldValues` is part of link:kafka-streams-internals-KStreamAggProcessorSupplier.adoc#enableSendingOldValues[KStreamAggProcessorSupplier Contract] to...FIXME. 43 | 44 | `enableSendingOldValues` simply turns the <> flag on. 45 | 46 | === [[mergeSessionWindow]] `mergeSessionWindow` Internal Method 47 | 48 | [source, java] 49 | ---- 50 | SessionWindow mergeSessionWindow(final SessionWindow one, final SessionWindow two) 51 | ---- 52 | 53 | `mergeSessionWindow`...FIXME 54 | 55 | NOTE: `mergeSessionWindow` is used when...FIXME 56 | -------------------------------------------------------------------------------- /kafka-streams-internals-KStreamTransform.adoc: -------------------------------------------------------------------------------- 1 | == [[KStreamTransform]] KStreamTransform -- Supplier of KStreamTransformProcessors 2 | 3 | `KStreamTransform` is a <> of <>. 4 | 5 | `KStreamTransform` is <> exclusively when `KStreamImpl` is requested to <>. 6 | 7 | [[transformerSupplier]] 8 | [[creating-instance]] 9 | `KStreamTransform` takes a single <> when created. 10 | 11 | [[get]] 12 | When <>, `KStreamTransform` creates a new <> with a new <> (after <> from the <>). 13 | -------------------------------------------------------------------------------- /kafka-streams-internals-KStreamTransformProcessor.adoc: -------------------------------------------------------------------------------- 1 | == [[KStreamTransformProcessor]] KStreamTransformProcessor 2 | 3 | `KStreamTransformProcessor` is a <> that is used for <> stateful operator. 4 | 5 | `KStreamTransformProcessor` is <> when `KStreamTransform` is requested for a <>. 6 | 7 | [[transformer]] 8 | [[creating-instance]] 9 | `KStreamTransformProcessor` takes a single <> when created. 10 | 11 | === [[close]] Closing Processor -- `close` Method 12 | 13 | [source, java] 14 | ---- 15 | void close() 16 | ---- 17 | 18 | NOTE: `close` is part of the <> to close the <>. 19 | 20 | `close`...FIXME 21 | 22 | === [[init]] Initializing Processor -- `init` Method 23 | 24 | [source, java] 25 | ---- 26 | void init(ProcessorContext context) 27 | ---- 28 | 29 | NOTE: `init` is part of the <> to initialize the <>. 30 | 31 | `init`...FIXME 32 | 33 | === [[process]] Processing Record -- `process` Method 34 | 35 | [source, java] 36 | ---- 37 | void process(K1 key, V1 value) 38 | ---- 39 | 40 | NOTE: `process` is part of the <> to process a single record. 41 | 42 | `process`...FIXME 43 | -------------------------------------------------------------------------------- /kafka-streams-internals-KStreamTransformValues.adoc: -------------------------------------------------------------------------------- 1 | == [[KStreamTransformValues]] KStreamTransformValues -- Supplier of KStreamTransformValuesProcessors 2 | 3 | `KStreamTransformValues` is a link:kafka-streams-ProcessorSupplier.adoc[ProcessorSupplier] that <>. 4 | 5 | `KStreamTransformValues` is <> exclusively when `KStreamImpl` is requested to link:kafka-streams-internals-KStreamImpl.adoc#transformValues[transform values with optional state] (aka _stateful record-by-record value transformation_). 6 | 7 | [[creating-instance]] 8 | [[valueTransformerSupplier]] 9 | `KStreamTransformValues` takes a `InternalValueTransformerWithKeySupplier` when created. 10 | 11 | === [[get]] `get` Method 12 | 13 | [source, java] 14 | ---- 15 | Processor get() 16 | ---- 17 | 18 | NOTE: `get` is part of link:kafka-streams-ProcessorSupplier.adoc#get[ProcessorSupplier Contract] to supply a stream processor. 19 | 20 | `get`...FIXME 21 | -------------------------------------------------------------------------------- /kafka-streams-internals-KStreamTransformValuesProcessor.adoc: -------------------------------------------------------------------------------- 1 | == [[KStreamTransformValuesProcessor]] KStreamTransformValuesProcessor 2 | 3 | `KStreamTransformValuesProcessor` is a link:kafka-streams-Processor.adoc[stream processor] that...FIXME 4 | 5 | `KStreamTransformValuesProcessor` is <> exclusively when `KStreamTransformValues` is requested to link:kafka-streams-internals-KStreamTransformValues.adoc#get[supply a stream processor]. 6 | 7 | [[creating-instance]] 8 | [[valueTransformer]] 9 | `KStreamTransformValuesProcessor` takes a `InternalValueTransformerWithKey` when created. 10 | 11 | NOTE: `KStreamTransformValuesProcessor` is `public static` class of a link:kafka-streams-internals-KStreamTransformValues.adoc[KStreamTransformValues] processor supplier. 12 | 13 | === [[init]] Initializing Processor Node (Given ProcessorContext) -- `init` Method 14 | 15 | [source, java] 16 | ---- 17 | void init(final ProcessorContext context) 18 | ---- 19 | 20 | NOTE: `init` is part of link:kafka-streams-Processor.adoc#init[Processor Contract] to initialize a stream processor. 21 | 22 | `init`...FIXME 23 | -------------------------------------------------------------------------------- /kafka-streams-internals-KStreamWindowAggregate.adoc: -------------------------------------------------------------------------------- 1 | == [[KStreamWindowAggregate]] KStreamWindowAggregate 2 | 3 | `KStreamWindowAggregate` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-KStreamWindowAggregateProcessor.adoc: -------------------------------------------------------------------------------- 1 | == [[KStreamWindowAggregateProcessor]] KStreamWindowAggregateProcessor 2 | 3 | `KStreamWindowAggregateProcessor` is a <> that is used for...FIXME 4 | 5 | `KStreamWindowAggregateProcessor` is <> when...FIXME 6 | 7 | [[creating-instance]] 8 | `KStreamWindowAggregateProcessor` takes no arguments when created. 9 | 10 | === [[init]] Initializing Processor -- `init` Method 11 | 12 | [source, java] 13 | ---- 14 | void init(final ProcessorContext context) 15 | ---- 16 | 17 | NOTE: `init` is part of the <> to initialize the <>. 18 | 19 | `init`...FIXME 20 | 21 | === [[process]] Processing Record -- `process` Method 22 | 23 | [source, java] 24 | ---- 25 | void process(final K key, final V value) 26 | ---- 27 | 28 | NOTE: `process` is part of the <> to process a single record. 29 | 30 | `process`...FIXME 31 | -------------------------------------------------------------------------------- /kafka-streams-internals-KTableKTableAbstractJoinValueGetterSupplier.adoc: -------------------------------------------------------------------------------- 1 | == [[KTableKTableAbstractJoinValueGetterSupplier]] KTableKTableAbstractJoinValueGetterSupplier 2 | 3 | `KTableKTableAbstractJoinValueGetterSupplier` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-KTableMaterializedValueGetterSupplier.adoc: -------------------------------------------------------------------------------- 1 | == [[KTableMaterializedValueGetterSupplier]] KTableMaterializedValueGetterSupplier 2 | 3 | `KTableMaterializedValueGetterSupplier` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-KTableSource.adoc: -------------------------------------------------------------------------------- 1 | == [[KTableSource]] KTableSource -- ProcessorSupplier of KTableSourceProcessors 2 | 3 | `KTableSource` is a <> of <>. 4 | 5 | `KTableSource` is <> when `InternalStreamsBuilder` is requested to add a <> and a <> to a topology (when `StreamsBuilder` is requested to create a <> and <> for topics, respectively). 6 | 7 | [[storeName]] 8 | [[creating-instance]] 9 | `KTableSource` takes a store name when created. 10 | 11 | [[get]] 12 | `KTableSource` creates a new <> whenever requested to <>. 13 | -------------------------------------------------------------------------------- /kafka-streams-internals-KTableSourceProcessor.adoc: -------------------------------------------------------------------------------- 1 | == [[KTableSourceProcessor]] KTableSourceProcessor 2 | 3 | `KTableSourceProcessor` is a <> that is <> when `KTableSource` is requested to <>. 4 | 5 | [[creating-instance]] 6 | `KTableSourceProcessor` takes no arguments when created. 7 | 8 | [[init]] 9 | When requested to <>, `KTableSourceProcessor`...FIXME 10 | 11 | [[process]] 12 | When requested to <>, `KTableSourceProcessor`...FIXME 13 | -------------------------------------------------------------------------------- /kafka-streams-internals-KTableSourceValueGetterSupplier.adoc: -------------------------------------------------------------------------------- 1 | == [[KTableSourceValueGetterSupplier]] KTableSourceValueGetterSupplier 2 | 3 | `KTableSourceValueGetterSupplier` is a link:kafka-streams-internals-KTableValueGetterSupplier.adoc[KTableValueGetterSupplier] that...FIXME 4 | 5 | `KTableSourceValueGetterSupplier` is <> when...FIXME 6 | 7 | [[storeName]] 8 | [[creating-instance]] 9 | `KTableSourceValueGetterSupplier` takes the store name when created. 10 | -------------------------------------------------------------------------------- /kafka-streams-internals-KTableSuppressProcessor.adoc: -------------------------------------------------------------------------------- 1 | == [[KTableSuppressProcessor]] KTableSuppressProcessor 2 | 3 | `KTableSuppressProcessor` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-KTableValueGetter.adoc: -------------------------------------------------------------------------------- 1 | == [[KTableValueGetter]] KTableValueGetter 2 | 3 | `KTableValueGetter` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-KTableValueGetterSupplier.adoc: -------------------------------------------------------------------------------- 1 | == [[KTableValueGetterSupplier]] KTableValueGetterSupplier 2 | 3 | `KTableValueGetterSupplier` is the <> of <> that can <> for <>. 4 | 5 | [[contract]] 6 | [source, java] 7 | ---- 8 | package org.apache.kafka.streams.kstream.internals; 9 | 10 | interface KTableValueGetterSupplier { 11 | KTableValueGetter get(); 12 | String[] storeNames(); 13 | } 14 | ---- 15 | 16 | .KTableValueGetterSupplier Contract 17 | [cols="1,2",options="header",width="100%"] 18 | |=== 19 | | Method 20 | | Description 21 | 22 | | `get` 23 | | [[get]] Used when...FIXME 24 | 25 | | `storeNames` 26 | | [[storeNames]] Used when...FIXME 27 | |=== 28 | 29 | [[implementations]] 30 | .KTableValueGetterSuppliers 31 | [cols="1,2",options="header",width="100%"] 32 | |=== 33 | | KTableValueGetterSupplier 34 | | Description 35 | 36 | | link:kafka-streams-internals-KTableMaterializedValueGetterSupplier.adoc[KTableMaterializedValueGetterSupplier] 37 | | [[KTableMaterializedValueGetterSupplier]] 38 | 39 | | link:kafka-streams-internals-KTableSourceValueGetterSupplier.adoc[KTableSourceValueGetterSupplier] 40 | | [[KTableSourceValueGetterSupplier]] 41 | 42 | | link:kafka-streams-internals-KTableKTableAbstractJoinValueGetterSupplier.adoc[KTableKTableAbstractJoinValueGetterSupplier] 43 | | [[KTableKTableAbstractJoinValueGetterSupplier]] 44 | |=== 45 | -------------------------------------------------------------------------------- /kafka-streams-internals-KeyValueToTimestampedKeyValueByteStoreAdapter.adoc: -------------------------------------------------------------------------------- 1 | == [[KeyValueToTimestampedKeyValueByteStoreAdapter]] KeyValueToTimestampedKeyValueByteStoreAdapter 2 | 3 | `KeyValueToTimestampedKeyValueByteStoreAdapter` is a concrete <> that...FIXME 4 | 5 | [[persistent]] 6 | `KeyValueToTimestampedKeyValueByteStoreAdapter` is always <>. 7 | -------------------------------------------------------------------------------- /kafka-streams-internals-MaterializedInternal.adoc: -------------------------------------------------------------------------------- 1 | == [[MaterializedInternal]] MaterializedInternal 2 | 3 | `MaterializedInternal` is a <> that...FIXME 4 | 5 | [[queriable]] 6 | [[isQueryable]] 7 | `MaterializedInternal` can be *queriable* which is...FIXME 8 | 9 | [[loggingEnabled]] 10 | `MaterializedInternal` uses the `loggingEnabled` flag for...FIXME 11 | 12 | [[cachingEnabled]] 13 | `MaterializedInternal` uses the `cachingEnabled` flag for...FIXME 14 | 15 | [[logConfig]] 16 | [[topicConfig]] 17 | `MaterializedInternal` uses the `topicConfig` registry with mappings for...FIXME 18 | 19 | `MaterializedInternal` allows accessing the optional <>. 20 | 21 | [[storeName]] 22 | `MaterializedInternal` uses the <> for...FIXME 23 | 24 | === [[generateStoreNameIfNeeded]] `generateStoreNameIfNeeded` Method 25 | 26 | [source, java] 27 | ---- 28 | void generateStoreNameIfNeeded( 29 | final InternalNameProvider nameProvider, 30 | final String generatedStorePrefix) 31 | ---- 32 | 33 | `generateStoreNameIfNeeded`...FIXME 34 | 35 | NOTE: `generateStoreNameIfNeeded` is used when...FIXME 36 | 37 | === [[withLoggingDisabled]] `withLoggingDisabled` Method 38 | 39 | [source, java] 40 | ---- 41 | Materialized withLoggingDisabled() 42 | ---- 43 | 44 | `withLoggingDisabled` turns the <> flag off and removes all of the mappings from the <>. 45 | 46 | In the end, `withLoggingDisabled` returns the current `Materialized` instance. 47 | 48 | NOTE: `withLoggingDisabled` is used exclusively when `InternalStreamsBuilder` is requested to <>. 49 | 50 | === [[storeSupplier]] Accessing Optional StoreSupplier -- `storeSupplier` Method 51 | 52 | [source, java] 53 | ---- 54 | StoreSupplier storeSupplier() 55 | ---- 56 | 57 | `storeSupplier` simply returns the <>. 58 | 59 | NOTE: `storeSupplier` is used when <>, <>, and <> are requested to materialize a StoreBuilder. 60 | -------------------------------------------------------------------------------- /kafka-streams-internals-MemoryLRUCache.adoc: -------------------------------------------------------------------------------- 1 | == [[MemoryLRUCache]] MemoryLRUCache 2 | 3 | `MemoryLRUCache` is...FIXME 4 | 5 | === [[init]] Initialize State Store -- `init` Method 6 | 7 | [source, java] 8 | ---- 9 | void init(final ProcessorContext context, final StateStore root) 10 | ---- 11 | 12 | NOTE: `init` is part of <> to initialize the <>. 13 | 14 | `init`...FIXME 15 | -------------------------------------------------------------------------------- /kafka-streams-internals-MeteredKeyValueBytesStore.adoc: -------------------------------------------------------------------------------- 1 | == [[MeteredKeyValueBytesStore]] MeteredKeyValueBytesStore 2 | 3 | `MeteredKeyValueBytesStore` is a concrete link:kafka-streams-KeyValueStore.adoc[KeyValueStore] that...FIXME 4 | 5 | === [[init]] `init` Method 6 | 7 | [source, java] 8 | ---- 9 | FIXME 10 | ---- 11 | 12 | NOTE: `init` is part of link:LINK#init[HERE Contract] to...FIXME. 13 | 14 | `init`...FIXME 15 | -------------------------------------------------------------------------------- /kafka-streams-internals-MeteredKeyValueStore.adoc: -------------------------------------------------------------------------------- 1 | == [[MeteredKeyValueStore]] MeteredKeyValueStore 2 | 3 | `MeteredKeyValueStore` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-MeteredSessionStore.adoc: -------------------------------------------------------------------------------- 1 | == [[MeteredSessionStore]] MeteredSessionStore 2 | 3 | `MeteredSessionStore` is a <> that...FIXME 4 | 5 | === [[init]] Initialize State Store -- `init` Method 6 | 7 | [source, java] 8 | ---- 9 | void init(final ProcessorContext context, final StateStore root) 10 | ---- 11 | 12 | NOTE: `init` is part of...FIXME 13 | 14 | `init`...FIXME 15 | 16 | === [[put]] Storing Aggregated Value for Session -- `put` Method 17 | 18 | [source, java] 19 | ---- 20 | void put(final Windowed sessionKey, final V aggregate) 21 | ---- 22 | 23 | NOTE: `put` is part of link:kafka-streams-SessionStore.adoc#put[SessionStore Contract] to store an aggregated value for a session. 24 | 25 | `put`...FIXME 26 | 27 | === [[remove]] Removing Aggregated Value for Session -- `remove` Method 28 | 29 | [source, java] 30 | ---- 31 | void remove(final Windowed sessionKey) 32 | ---- 33 | 34 | NOTE: `remove` is part of link:kafka-streams-SessionStore.adoc#remove[SessionStore Contract] to remove an aggregated value for a session. 35 | 36 | `remove`...FIXME 37 | -------------------------------------------------------------------------------- /kafka-streams-internals-MeteredWindowStore.adoc: -------------------------------------------------------------------------------- 1 | == [[MeteredWindowStore]] MeteredWindowStore -- State Store of WindowStoreBuilder 2 | 3 | `MeteredWindowStore` is a concrete <> that registers the <> (when <>). 4 | 5 | `MeteredWindowStore` is <> exclusively when `WindowStoreBuilder` is requested for a <>. 6 | 7 | [[metrics]] 8 | .MeteredWindowStore's (Latency and Throughput) Metric Sensors 9 | [cols="1m,1m,1m,2",options="header",width="100%"] 10 | |=== 11 | | Metric Sensor 12 | | Name 13 | | RecordingLevel 14 | | Description 15 | 16 | | putTime 17 | | put 18 | | DEBUG 19 | | [[putTime]] Latency of <> 20 | 21 | | fetchTime 22 | | fetch 23 | | DEBUG 24 | | [[fetchTime]] Latency of <>, <> and <> 25 | 26 | | flushTime 27 | | flush 28 | | DEBUG 29 | | [[flushTime]] Latency of <> 30 | 31 | | restoreTime 32 | | restore 33 | | DEBUG 34 | | [[restoreTime]] Latency of <> 35 | |=== 36 | 37 | NOTE: All the <> are at `DEBUG` recording level so you have to turn them on using <> streams property. 38 | 39 | === [[creating-instance]] Creating MeteredWindowStore Instance 40 | 41 | `MeteredWindowStore` takes the following when created: 42 | 43 | * [[inner]] <> 44 | * [[metricScope]] Scope name for the <> (aka _metric scope_) 45 | * [[time]] `Time` 46 | * [[keySerde]] `Serde` for keys 47 | * [[valueSerde]] `Serde` for values 48 | 49 | `MeteredWindowStore` initializes the <>. 50 | 51 | === [[init]] Initializing State Store -- `init` Method 52 | 53 | [source, java] 54 | ---- 55 | void init(final ProcessorContext context, final StateStore root) 56 | ---- 57 | 58 | NOTE: `init` is part of the <> to initializes the state store. 59 | 60 | `init`...FIXME 61 | 62 | `init` requests the `ProcessorContext` for the <> and registers the <>. 63 | 64 | `init`...FIXME 65 | -------------------------------------------------------------------------------- /kafka-streams-internals-NamedCache.adoc: -------------------------------------------------------------------------------- 1 | == [[NamedCache]] NamedCache 2 | 3 | `NamedCache` is...FIXME 4 | 5 | `NamedCache` is <> exclusively when `ThreadCache` is requested to <>. 6 | 7 | === [[creating-instance]] Creating NamedCache Instance 8 | 9 | `NamedCache` takes the following to be created: 10 | 11 | * [[name]] Name 12 | * [[metrics]] <> 13 | 14 | `NamedCache` initializes the <>. 15 | 16 | === [[NamedCacheMetrics]] NamedCacheMetrics -- Performance Metrics of NamedCache 17 | 18 | `NamedCache` uses the following performance metrics...FIXME 19 | 20 | === [[setListener]] `setListener` Method 21 | 22 | [source, java] 23 | ---- 24 | void setListener( 25 | ThreadCache.DirtyEntryFlushListener listener) 26 | ---- 27 | 28 | `setListener` simply sets the <> internal registry to be the `ThreadCache.DirtyEntryFlushListener`. 29 | 30 | NOTE: `setListener` is used exclusively when `ThreadCache` is requested to <>. 31 | 32 | === [[evict]] `evict` Method 33 | 34 | [source, java] 35 | ---- 36 | void evict() 37 | ---- 38 | 39 | `evict`...FIXME 40 | 41 | NOTE: `evict` is used exclusively when `ThreadCache` is requested to <> 42 | 43 | === [[close]] `close` Method 44 | 45 | [source, java] 46 | ---- 47 | void close() 48 | ---- 49 | 50 | `close`...FIXME 51 | 52 | NOTE: `close` is used when...FIXME 53 | 54 | === [[flush]] `flush` Method 55 | 56 | [source, java] 57 | ---- 58 | void flush() // <1> 59 | // Private API 60 | void flush(LRUNode evicted) 61 | ---- 62 | <1> Uses `null` for the evicted `LRUNode` 63 | 64 | `flush`...FIXME 65 | 66 | NOTE: `flush` is used when...FIXME 67 | 68 | === [[internal-properties]] Internal Properties 69 | 70 | [cols="30m,70",options="header",width="100%"] 71 | |=== 72 | | Name 73 | | Description 74 | 75 | | listener 76 | a| [[listener]] `ThreadCache.DirtyEntryFlushListener` 77 | 78 | * Initialized in <> 79 | 80 | * Removed (_nullified_) in <> 81 | 82 | Used exclusively when <> 83 | 84 | |=== 85 | -------------------------------------------------------------------------------- /kafka-streams-internals-NodeMetrics.adoc: -------------------------------------------------------------------------------- 1 | == [[NodeMetrics]] NodeMetrics 2 | 3 | `NodeMetrics` is...FIXME 4 | 5 | `NodeMetrics` is <> exclusively when `ProcessorNode` is requested to link:kafka-streams-internals-ProcessorNode.adoc#init[init]. 6 | 7 | [[internal-registries]] 8 | .NodeMetrics's Internal Properties (e.g. Registries, Counters and Flags) 9 | [cols="1,2",options="header",width="100%"] 10 | |=== 11 | | Name 12 | | Description 13 | 14 | | `nodeProcessTimeSensor` 15 | | [[nodeProcessTimeSensor]] Kafka `Sensor` 16 | 17 | Used when...FIXME 18 | 19 | | `nodePunctuateTimeSensor` 20 | | [[nodePunctuateTimeSensor]] Kafka `Sensor` 21 | 22 | Used when...FIXME 23 | 24 | | `sourceNodeForwardSensor` 25 | | [[sourceNodeForwardSensor]] Kafka `Sensor` 26 | 27 | Used when...FIXME 28 | 29 | | `sourceNodeSkippedDueToDeserializationError` 30 | | [[sourceNodeSkippedDueToDeserializationError]] Kafka `Sensor` 31 | 32 | Used when...FIXME 33 | 34 | | `nodeCreationSensor` 35 | | [[nodeCreationSensor]] Kafka `Sensor` 36 | 37 | Used when...FIXME 38 | 39 | | `nodeDestructionSensor` 40 | | [[nodeDestructionSensor]] Kafka `Sensor` 41 | 42 | Used when...FIXME 43 | |=== 44 | 45 | === [[creating-instance]] Creating NodeMetrics Instance 46 | 47 | `NodeMetrics` takes the following when created: 48 | 49 | * [[metrics]] link:kafka-streams-StreamsMetrics.adoc[StreamsMetrics] 50 | * [[name]] link:kafka-streams-internals-ProcessorNode.adoc#name[Name] of a `ProcessorNode` 51 | * [[context]] link:kafka-streams-ProcessorContext.adoc[ProcessorContext] 52 | 53 | `NodeMetrics` initializes the <>. 54 | -------------------------------------------------------------------------------- /kafka-streams-internals-OptimizableRepartitionNode.adoc: -------------------------------------------------------------------------------- 1 | == [[OptimizableRepartitionNode]] OptimizableRepartitionNode 2 | 3 | `OptimizableRepartitionNode` is...FIXME 4 | 5 | === [[writeToTopology]] `writeToTopology` Method 6 | 7 | [source, java] 8 | ---- 9 | void writeToTopology(final InternalTopologyBuilder topologyBuilder) 10 | ---- 11 | 12 | NOTE: `writeToTopology` is part of the <> to...FIXME. 13 | 14 | `writeToTopology`...FIXME 15 | -------------------------------------------------------------------------------- /kafka-streams-internals-ProcessorNodePunctuator.adoc: -------------------------------------------------------------------------------- 1 | == [[ProcessorNodePunctuator]] ProcessorNodePunctuator Contract 2 | 3 | `ProcessorNodePunctuator` is the <> of <> that can <>, i.e. execute scheduled periodic actions. 4 | 5 | [[contract]] 6 | [[punctuate]] 7 | [source, java] 8 | ---- 9 | void punctuate( 10 | ProcessorNode node, 11 | long streamTime, 12 | PunctuationType type, 13 | Punctuator punctuator) 14 | ---- 15 | 16 | `punctuate` is used exclusively when `PunctuationQueue` is requested to <> (for `StreamTask` that attempts to punctuate by <> and <> time). 17 | 18 | [[implementations]] 19 | NOTE: <> is the default and only known implementation of the <> in Kafka Streams. 20 | -------------------------------------------------------------------------------- /kafka-streams-internals-ProcessorParameters.adoc: -------------------------------------------------------------------------------- 1 | == [[ProcessorParameters]] ProcessorParameters 2 | 3 | [[processorSupplier]][[processorName]][[creating-instance]] 4 | `ProcessorParameters` represents a <> and a processor name to use to <> (to <>). 5 | 6 | `ProcessorParameters` abstracts away different number of processors of <> (i.e. <>, <>, <>, <>, <>, <>, <>, and <>) when requested to <> (when `InternalStreamsBuilder` is requested to <> when `StreamsBuilder` is requested to <>). 7 | -------------------------------------------------------------------------------- /kafka-streams-internals-ProcessorRecordContext.adoc: -------------------------------------------------------------------------------- 1 | == [[ProcessorRecordContext]] ProcessorRecordContext -- Record Metadata 2 | 3 | `ProcessorRecordContext` is a `RecordContext` that is the metadata of a record: 4 | 5 | * [[timestamp]] Timestamp 6 | * [[offset]] Offset 7 | * [[partition]] Partition 8 | * [[topic]] Topic 9 | 10 | [[creating-instance]] 11 | `ProcessorRecordContext` is created when...FIXME 12 | -------------------------------------------------------------------------------- /kafka-streams-internals-ProducedInternal.adoc: -------------------------------------------------------------------------------- 1 | == [[ProducedInternal]] ProducedInternal -- Internal Accessors to Produced Metadata 2 | 3 | `ProducedInternal` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-ProducerSupplier.adoc: -------------------------------------------------------------------------------- 1 | == [[ProducerSupplier]] ProducerSupplier 2 | 3 | `ProducerSupplier` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-PunctuationQueue.adoc: -------------------------------------------------------------------------------- 1 | == [[PunctuationQueue]] PunctuationQueue 2 | 3 | [[pq]] 4 | `PunctuationQueue` manages a https://docs.oracle.com/en/java/javase/11/docs/api/java.base/java/util/PriorityQueue.html[java.util.PriorityQueue] of <>. 5 | 6 | === [[mayPunctuate]] Attempting to Punctuate -- `mayPunctuate` Method 7 | 8 | [source, java] 9 | ---- 10 | boolean mayPunctuate( 11 | long timestamp, 12 | PunctuationType type, 13 | ProcessorNodePunctuator processorNodePunctuator) 14 | ---- 15 | 16 | `mayPunctuate` takes the <> off the <> for which the <> is older (_smaller_) than the given `timestamp`. 17 | 18 | `mayPunctuate` then requests the given <> to <> (with the <> and the <> of every `PunctuationSchedule`, the given `timestamp` and the `PunctuationType`). 19 | 20 | In the end, `mayPunctuate` returns whether a `PunctuationSchedule` was punctuated (`true`) or not (`false`). 21 | 22 | NOTE: `mayPunctuate` is used when `StreamTask` is requested to attempt to punctuate by <> and <> time (with itself as the <>). 23 | 24 | == [[schedule]] Scheduling Cancellable Periodic Action (Punctuator) -- `schedule` Method 25 | 26 | [source, java] 27 | ---- 28 | Cancellable schedule(PunctuationSchedule sched) 29 | ---- 30 | 31 | `schedule`...FIXME 32 | 33 | NOTE: `schedule` is used when...FIXME 34 | -------------------------------------------------------------------------------- /kafka-streams-internals-QueryableStoreProvider.adoc: -------------------------------------------------------------------------------- 1 | == [[QueryableStoreProvider]] QueryableStoreProvider 2 | 3 | `QueryableStoreProvider` is <> along with link:kafka-streams-KafkaStreams.adoc#queryableStoreProvider[KafkaStreams] to <> when requested to link:kafka-streams-KafkaStreams.adoc#store[store]. 4 | 5 | === [[creating-instance]] Creating QueryableStoreProvider Instance 6 | 7 | `QueryableStoreProvider` takes the following when created: 8 | 9 | * [[storeProviders]] link:kafka-streams-internals-StateStoreProvider.adoc[StateStoreProviders] 10 | * [[globalStateStoreProvider]] link:kafka-streams-internals-GlobalStateStoreProvider.adoc[GlobalStateStoreProvider] 11 | 12 | === [[getStore]] `getStore` Method 13 | 14 | [source, java] 15 | ---- 16 | T getStore(final String storeName, final QueryableStoreType queryableStoreType) 17 | ---- 18 | 19 | `getStore`...FIXME 20 | 21 | NOTE: `getStore` is used exclusively when `KafkaStreams` is requested to link:kafka-streams-KafkaStreams.adoc#store[store]. 22 | -------------------------------------------------------------------------------- /kafka-streams-internals-QuickUnion.adoc: -------------------------------------------------------------------------------- 1 | == [[QuickUnion]] QuickUnion 2 | 3 | `QuickUnion` (of elements of type `T`) is...FIXME 4 | 5 | === [[root]] `root` Method 6 | 7 | [source, java] 8 | ---- 9 | T root(T id) 10 | ---- 11 | 12 | `root`...FIXME 13 | 14 | NOTE: `root` is used when...FIXME 15 | 16 | === [[add]] `add` Method 17 | 18 | [source, java] 19 | ---- 20 | void add(T id) 21 | ---- 22 | 23 | `add`...FIXME 24 | 25 | NOTE: `add` is used when...FIXME 26 | 27 | === [[unite]] `unite` Method 28 | 29 | [source, java] 30 | ---- 31 | void unite(T id1, T... idList) 32 | ---- 33 | 34 | `unite`...FIXME 35 | 36 | NOTE: `unite` is used when...FIXME 37 | -------------------------------------------------------------------------------- /kafka-streams-internals-RecordBatchingStateRestoreCallback.adoc: -------------------------------------------------------------------------------- 1 | == [[RecordBatchingStateRestoreCallback]] RecordBatchingStateRestoreCallback 2 | 3 | `RecordBatchingStateRestoreCallback` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-RecordDeserializer.adoc: -------------------------------------------------------------------------------- 1 | == [[RecordDeserializer]] RecordDeserializer 2 | 3 | [[creating-instance]] 4 | `RecordDeserializer` is the metadata of a <> that knows how to <>. 5 | 6 | `RecordDeserializer` takes the following when created: 7 | 8 | * [[sourceNode]] link:kafka-streams-internals-SourceNode.adoc[SourceNode] 9 | * [[deserializationExceptionHandler]] `DeserializationExceptionHandler` 10 | * [[logContext]] `LogContext` 11 | 12 | === [[deserialize]] Deserializing Kafka ConsumerRecord (in ProcessorContext) -- `deserialize` Method 13 | 14 | [source, java] 15 | ---- 16 | ConsumerRecord deserialize( 17 | final ProcessorContext processorContext, 18 | final ConsumerRecord rawRecord) 19 | ---- 20 | 21 | `deserialize`...FIXME 22 | 23 | NOTE: `deserialize` is used when...FIXME 24 | -------------------------------------------------------------------------------- /kafka-streams-internals-RecordInfo.adoc: -------------------------------------------------------------------------------- 1 | == [[RecordInfo]] RecordInfo 2 | 3 | `RecordInfo` is...FIXME 4 | 5 | === [[node]] Getting Processor Node -- `node` Method 6 | 7 | [source, java] 8 | ---- 9 | ProcessorNode node() 10 | ---- 11 | 12 | `node`...FIXME 13 | 14 | NOTE: `node` is used when...FIXME 15 | 16 | === [[queue]] Getting RecordQueue -- `queue` Method 17 | 18 | [source, java] 19 | ---- 20 | RecordQueue queue() 21 | ---- 22 | 23 | `queue`...FIXME 24 | 25 | NOTE: `queue` is used when...FIXME 26 | 27 | === [[partition]] Getting Topic Partition -- `partition` Method 28 | 29 | [source, java] 30 | ---- 31 | TopicPartition partition() 32 | ---- 33 | 34 | `partition`...FIXME 35 | 36 | NOTE: `partition` is used when...FIXME 37 | -------------------------------------------------------------------------------- /kafka-streams-internals-RestoringTasks.adoc: -------------------------------------------------------------------------------- 1 | == [[RestoringTasks]] RestoringTasks Contract 2 | 3 | `RestoringTasks` is the <> of <> that can <>. 4 | 5 | [[contract]] 6 | .RestoringTasks Contract 7 | [cols="30m,70",options="header",width="100%"] 8 | |=== 9 | | Method 10 | | Description 11 | 12 | | restoringTaskFor 13 | a| [[restoringTaskFor]] 14 | 15 | [source, java] 16 | ---- 17 | StreamTask restoringTaskFor(TopicPartition partition) 18 | ---- 19 | 20 | Finds the restoring <> for a partition 21 | 22 | Used exclusively when `StoreChangelogReader` is requested to <> 23 | 24 | |=== 25 | 26 | [[implementations]] 27 | NOTE: <> is the default and only known implementation of the <> in Kafka Streams. 28 | -------------------------------------------------------------------------------- /kafka-streams-internals-RocksDBSegmentedBytesStore.adoc: -------------------------------------------------------------------------------- 1 | == [[RocksDBSegmentedBytesStore]] RocksDBSegmentedBytesStore 2 | 3 | `RocksDBSegmentedBytesStore` is a concrete <> (as a <> of `KeyValueSegments`) that...FIXME 4 | 5 | `RocksDBSegmentedBytesStore` is <> when: 6 | 7 | * `RocksDbSessionBytesStoreSupplier` is requested for a <> 8 | 9 | * `RocksDbWindowBytesStoreSupplier` is requested for a <> 10 | 11 | === [[creating-instance]] Creating RocksDBSegmentedBytesStore Instance 12 | 13 | `RocksDBSegmentedBytesStore` takes the following to be created: 14 | 15 | * [[name]] Name 16 | * [[metricScope]] Metric scope 17 | * [[retention]] `retention` 18 | * [[segmentInterval]] `segmentInterval` 19 | * [[keySchema]] `KeySchema` 20 | -------------------------------------------------------------------------------- /kafka-streams-internals-RocksDBSessionStore.adoc: -------------------------------------------------------------------------------- 1 | == [[RocksDBSessionStore]] RocksDBSessionStore 2 | 3 | `RocksDBSessionStore` is...FIXME 4 | 5 | === [[init]] Initialize State Store -- `init` Method 6 | 7 | [source, java] 8 | ---- 9 | void init(final ProcessorContext context, final StateStore root) 10 | ---- 11 | 12 | NOTE: `init` is part of...FIXME 13 | 14 | `init`...FIXME 15 | -------------------------------------------------------------------------------- /kafka-streams-internals-RocksDBStore.adoc: -------------------------------------------------------------------------------- 1 | == [[RocksDBStore]] RocksDBStore 2 | 3 | `RocksDBStore` is a concrete <> that...FIXME 4 | 5 | [[persistent]] 6 | `RocksDBStore` is always <>. 7 | 8 | === [[init]] Initialize State Store -- `init` Method 9 | 10 | [source, java] 11 | ---- 12 | void init(final ProcessorContext context, final StateStore root) 13 | ---- 14 | 15 | NOTE: `init` is part of <> to initialize the <>. 16 | 17 | `init`...FIXME 18 | -------------------------------------------------------------------------------- /kafka-streams-internals-RocksDBTimestampedStore.adoc: -------------------------------------------------------------------------------- 1 | == [[RocksDBTimestampedStore]] RocksDBTimestampedStore 2 | 3 | `RocksDBTimestampedStore` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-RocksDBWindowStore.adoc: -------------------------------------------------------------------------------- 1 | == [[RocksDBWindowStore]] RocksDBWindowStore 2 | 3 | `RocksDBWindowStore` is a custom <> that uses the <> (as the underlying <>). 4 | 5 | `RocksDBWindowStore` is <> exclusively when `RocksDbWindowBytesStoreSupplier` is requested for a <> (and gives a `RocksDBWindowStore` with a <>). 6 | 7 | === [[creating-instance]] Creating RocksDBWindowStore Instance 8 | 9 | `RocksDBWindowStore` takes the following when created: 10 | 11 | * [[bytesStore]] <> 12 | * [[keySerde]] `Serde` for keys 13 | * [[valueSerde]] `Serde` for values 14 | * [[retainDuplicates]] `retainDuplicates` flag 15 | * [[windowSize]] `windowSize` 16 | 17 | `RocksDBWindowStore` initializes the <>. 18 | 19 | === [[init]] `init` Method 20 | 21 | [source, java] 22 | ---- 23 | void init(final ProcessorContext context, final StateStore root) 24 | ---- 25 | 26 | NOTE: `init` is part of the <> to initialize the state store. 27 | 28 | `init`...FIXME 29 | -------------------------------------------------------------------------------- /kafka-streams-internals-RocksDbKeyValueBytesStoreSupplier.adoc: -------------------------------------------------------------------------------- 1 | == [[RocksDbKeyValueBytesStoreSupplier]] RocksDbKeyValueBytesStoreSupplier 2 | 3 | `RocksDbKeyValueBytesStoreSupplier` is a <>. 4 | 5 | `RocksDbKeyValueBytesStoreSupplier` is <> when `Stores` utility is used to create persistent <>: 6 | 7 | * <> (with the <> off) 8 | 9 | * <> (with the <> on) 10 | 11 | [[metricsScope]] 12 | `RocksDbKeyValueBytesStoreSupplier` uses *rocksdb-state* metric scope for the <>. 13 | 14 | === [[creating-instance]] Creating RocksDbKeyValueBytesStoreSupplier Instance 15 | 16 | `RocksDbKeyValueBytesStoreSupplier` takes the following to be created: 17 | 18 | * [[name]] Name 19 | * [[returnTimestampedStore]] `returnTimestampedStore` flag 20 | 21 | === [[get]] Supplying KeyValueStore -- `get` Method 22 | 23 | [source, java] 24 | ---- 25 | KeyValueStore get() 26 | ---- 27 | 28 | NOTE: `get` is part of the <> to supply (_get_) a <>. 29 | 30 | `get` gives the following <>: 31 | 32 | * <> when <> is on (`true`) 33 | 34 | * <> when <> is off (`false`) 35 | -------------------------------------------------------------------------------- /kafka-streams-internals-RocksDbSessionBytesStoreSupplier.adoc: -------------------------------------------------------------------------------- 1 | == [[RocksDbSessionBytesStoreSupplier]] RocksDbSessionBytesStoreSupplier 2 | 3 | `RocksDbSessionBytesStoreSupplier` is a concrete <> of <> (`SessionStore`). 4 | 5 | `RocksDbSessionBytesStoreSupplier` is <> when `Stores` factory is requested to <>. 6 | 7 | === [[get]] Creating State Store -- `get` Method 8 | 9 | [source, java] 10 | ---- 11 | SessionStore get() 12 | ---- 13 | 14 | NOTE: `get` is part of the <> to get a <>. 15 | 16 | `get` simply creates a new <>. 17 | -------------------------------------------------------------------------------- /kafka-streams-internals-RocksDbWindowBytesStoreSupplier.adoc: -------------------------------------------------------------------------------- 1 | == [[RocksDbWindowBytesStoreSupplier]] RocksDbWindowBytesStoreSupplier 2 | 3 | `RocksDbWindowBytesStoreSupplier` is a concrete <> that <>. 4 | 5 | In fact, `RocksDbWindowBytesStoreSupplier` is the one and only known <>. 6 | 7 | [[metricsScope]] 8 | `RocksDbWindowBytesStoreSupplier` uses *rocksdb-window-state* as the <>. 9 | 10 | `RocksDbWindowBytesStoreSupplier` is <> when `Stores` factory is requested to <>. 11 | 12 | [source, scala] 13 | ---- 14 | import org.apache.kafka.streams.state.Stores 15 | val windowStore = Stores.persistentWindowStore("window-store-name", 1000, 2, 1000, true) 16 | 17 | scala> :type windowStore 18 | org.apache.kafka.streams.state.WindowBytesStoreSupplier 19 | 20 | import org.apache.kafka.streams.state.internals.RocksDbWindowBytesStoreSupplier 21 | assert(windowStore.isInstanceOf[RocksDbWindowBytesStoreSupplier]) 22 | ---- 23 | 24 | === [[get]] Creating State Store -- `get` Method 25 | 26 | [source, java] 27 | ---- 28 | WindowStore get() 29 | ---- 30 | 31 | NOTE: `get` is part of the <> to get a <>. 32 | 33 | `get` creates a <> (with the <>, the <> and the <>). 34 | 35 | `get` then returns a new <> (with the `RocksDBSegmentedBytesStore` and the <> and the <>). 36 | 37 | === [[creating-instance]] Creating RocksDbWindowBytesStoreSupplier Instance 38 | 39 | `RocksDbWindowBytesStoreSupplier` takes the following when created: 40 | 41 | * [[name]] Name 42 | * [[retentionPeriod]] `retentionPeriod` 43 | * [[segments]] Number of segments (must be `2` or higher) 44 | * [[windowSize]] `windowSize` 45 | * [[retainDuplicates]] `retainDuplicates` flag 46 | -------------------------------------------------------------------------------- /kafka-streams-internals-SegmentedBytesStore.adoc: -------------------------------------------------------------------------------- 1 | == [[SegmentedBytesStore]] SegmentedBytesStore Contract 2 | 3 | `SegmentedBytesStore` is the <> of the <> for <> that <>. 4 | 5 | [[contract]] 6 | .SegmentedBytesStore Contract 7 | [cols="30m,70",options="header",width="100%"] 8 | |=== 9 | | Method 10 | | Description 11 | 12 | | all 13 | a| [[all]] 14 | 15 | [source, java] 16 | ---- 17 | KeyValueIterator all() 18 | ---- 19 | 20 | Used when...FIXME 21 | 22 | | fetch 23 | a| [[fetch]] 24 | 25 | [source, java] 26 | ---- 27 | KeyValueIterator fetch( 28 | Bytes keyFrom, 29 | Bytes keyTo, 30 | long from, 31 | long to) 32 | KeyValueIterator fetch( 33 | Bytes key, 34 | long from, 35 | long to) 36 | ---- 37 | 38 | Used when...FIXME 39 | 40 | | fetchAll 41 | a| [[fetchAll]] 42 | 43 | [source, java] 44 | ---- 45 | KeyValueIterator fetchAll( 46 | long from, 47 | long to) 48 | ---- 49 | 50 | Used when...FIXME 51 | 52 | | get 53 | a| [[get]] 54 | 55 | [source, java] 56 | ---- 57 | byte[] get(Bytes key) 58 | ---- 59 | 60 | Used when...FIXME 61 | 62 | | put 63 | a| [[put]] 64 | 65 | [source, java] 66 | ---- 67 | void put( 68 | Bytes key, 69 | byte[] value) 70 | ---- 71 | 72 | Used when...FIXME 73 | 74 | | remove 75 | a| [[remove]] 76 | 77 | [source, java] 78 | ---- 79 | void remove(Bytes key) 80 | ---- 81 | 82 | Used when...FIXME 83 | 84 | |=== 85 | 86 | [[implementations]] 87 | NOTE: <> is the only known direct implementation of the <> in Kafka Streams. 88 | -------------------------------------------------------------------------------- /kafka-streams-internals-SessionStoreBuilder.adoc: -------------------------------------------------------------------------------- 1 | == [[SessionStoreBuilder]] SessionStoreBuilder 2 | 3 | `SessionStoreBuilder` is an link:kafka-streams-internals-AbstractStoreBuilder.adoc[AbstractStoreBuilder] that <> (possibly with <> and <>). 4 | 5 | `SessionStoreBuilder` is <> exclusively when `Stores` is requested to link:kafka-streams-Stores.adoc#sessionStoreBuilder[create one] (with `Time.SYSTEM` <>). 6 | 7 | === [[creating-instance]] Creating SessionStoreBuilder Instance 8 | 9 | `SessionStoreBuilder` takes the following when created: 10 | 11 | * [[storeSupplier]] link:kafka-streams-SessionBytesStoreSupplier.adoc[SessionBytesStoreSupplier] 12 | * [[keySerde]] `Serde` for keys 13 | * [[valueSerde]] `Serde` for values 14 | * [[time]] `Time` 15 | 16 | === [[build]] Building MeteredSessionStore (with Optional Caching and Logging) -- `build` Method 17 | 18 | [source, java] 19 | ---- 20 | SessionStore build() 21 | ---- 22 | 23 | NOTE: `build` is part of link:kafka-streams-StoreBuilder.adoc#build[StoreBuilder Contract] to build a link:kafka-streams-StateStore.adoc[StateStore]. 24 | 25 | `build`...FIXME 26 | 27 | === [[maybeWrapCaching]] `maybeWrapCaching` Internal Method 28 | 29 | [source, java] 30 | ---- 31 | SessionStore maybeWrapCaching(final SessionStore inner) 32 | ---- 33 | 34 | `maybeWrapCaching`...FIXME 35 | 36 | NOTE: `maybeWrapCaching` is used when...FIXME 37 | 38 | === [[maybeWrapLogging]] `maybeWrapLogging` Internal Method 39 | 40 | [source, java] 41 | ---- 42 | SessionStore maybeWrapLogging(final SessionStore inner) 43 | ---- 44 | 45 | `maybeWrapLogging`...FIXME 46 | 47 | NOTE: `maybeWrapLogging` is used when...FIXME 48 | -------------------------------------------------------------------------------- /kafka-streams-internals-SessionWindow.adoc: -------------------------------------------------------------------------------- 1 | == [[SessionWindow]] SessionWindow 2 | 3 | `SessionWindow` is a concrete link:kafka-streams-Window.adoc[Window] with a custom <>. 4 | 5 | `SessionWindow` is <> when: 6 | 7 | * `KStreamSessionWindowAggregate` is requested to link:kafka-streams-internals-KStreamSessionWindowAggregate.adoc#mergeSessionWindow[mergeSessionWindow] 8 | 9 | * `KStreamSessionWindowAggregateProcessor` is requested to link:kafka-streams-internals-KStreamSessionWindowAggregateProcessor.adoc#process[process a single record] 10 | 11 | * `SessionKeySchema` is requested to `upperRangeFixedSize` and `lowerRangeFixedSize` 12 | 13 | === [[creating-instance]] Creating SessionWindow Instance 14 | 15 | `SessionWindow` takes the following when created: 16 | 17 | * [[startMs]] The start timestamp of the window (in milliseconds) 18 | * [[endMs]] The end timestamp of the window (in milliseconds) 19 | 20 | === [[overlap]] Checking If Two Windows Overlap -- `overlap` Method 21 | 22 | [source, java] 23 | ---- 24 | boolean overlap(final Window other) throws IllegalArgumentException 25 | ---- 26 | 27 | NOTE: `overlap` is part of link:kafka-streams-Window.adoc#overlap[Window Contract] to check whether a window overlaps with another. 28 | 29 | `overlap` compares the start and end timestampes and is positive (`true`) when link:kafka-streams-Window.adoc#endMs[end] timestamp of either window is exactly or greater than the link:kafka-streams-Window.adoc#startMs[start] timestamp of the other window. 30 | 31 | `overlap` throws a `IllegalArgumentException` when the `other` window is not a `SessionWindow`. 32 | 33 | ``` 34 | Cannot compare windows of different type. Other window has type [className]. 35 | ``` 36 | -------------------------------------------------------------------------------- /kafka-streams-internals-SourceNode.adoc: -------------------------------------------------------------------------------- 1 | == [[SourceNode]] SourceNode 2 | 3 | `SourceNode` is a link:kafka-streams-internals-ProcessorNode.adoc[ProcessorNode] that...FIXME 4 | 5 | `SourceNode` is <> exclusively when `SourceNodeFactory` is requested to link:kafka-streams-internals-InternalTopologyBuilder-SourceNodeFactory.adoc#build[build a processor node] (when...FIXME) 6 | 7 | [[context]] 8 | `SourceNode` gets a link:kafka-streams-ProcessorContext.adoc[ProcessorContext] when <>. It is later used when `SourceNode` <> (and simply forwards it downstream). 9 | 10 | === [[init]] `init` Method 11 | 12 | [source, java] 13 | ---- 14 | void init(ProcessorContext context) 15 | ---- 16 | 17 | NOTE: `init` is part of link:kafka-streams-internals-ProcessorNode.adoc#init[ProcessorNode Contract] to...FIXME. 18 | 19 | `init`...FIXME 20 | 21 | === [[process]] Processing Record -- `process` Method 22 | 23 | [source, java] 24 | ---- 25 | void process( 26 | K key, 27 | V value) 28 | ---- 29 | 30 | NOTE: `process` is part of link:kafka-streams-internals-ProcessorNode.adoc#process[ProcessorNode Contract] to...FIXME. 31 | 32 | `process` simply requests <> to link:kafka-streams-ProcessorContext.adoc#forward[forward] followed by informing the `sourceNodeForwardSensor` that a record was processed. 33 | 34 | === [[creating-instance]] Creating SourceNode Instance 35 | 36 | `SourceNode` takes the following when created: 37 | 38 | * [[name]] Node name 39 | * [[topics]] List of topics 40 | * [[timestampExtractor]] `TimestampExtractor` 41 | * [[keyDeserializer]] Key deserializer 42 | * [[valDeserializer]] Value deserializer 43 | 44 | `SourceNode` initializes the <>. 45 | -------------------------------------------------------------------------------- /kafka-streams-internals-StandbyContextImpl.adoc: -------------------------------------------------------------------------------- 1 | == [[StandbyContextImpl]] StandbyContextImpl 2 | 3 | `StandbyContextImpl` is...FIXME 4 | 5 | [[NO_OP_COLLECTOR]] 6 | `StandbyContextImpl` creates a <> that simply does nothing when executed (i.e. all methods are no-ops). 7 | -------------------------------------------------------------------------------- /kafka-streams-internals-StateDirectory.adoc: -------------------------------------------------------------------------------- 1 | == [[StateDirectory]] StateDirectory 2 | 3 | `StateDirectory` is...FIXME 4 | 5 | `StateDirectory` is <> when `KafkaStreams` is link:kafka-streams-KafkaStreams.adoc#stateDirectory[created]. 6 | 7 | [[stateDir]] 8 | `StateDirectory` uses link:kafka-streams-properties.adoc#state.dir[state.dir] and link:kafka-streams-properties.adoc#application.id[application.id] configuration properties for the location of the state store. 9 | 10 | NOTE: link:kafka-streams-properties.adoc#state.dir[state.dir] configuration property defaults to `/tmp/kafka-streams`. 11 | 12 | [[logging]] 13 | [TIP] 14 | ==== 15 | Enable `ALL` logging level for `org.apache.kafka.streams.processor.internals.StateDirectory` logger to see what happens inside. 16 | 17 | Add the following line to `log4j.properties`: 18 | 19 | ``` 20 | log4j.logger.org.apache.kafka.streams.processor.internals.StateDirectory=ALL 21 | ``` 22 | 23 | Refer to <>. 24 | ==== 25 | 26 | === [[clean]] `clean` Method 27 | 28 | [source, java] 29 | ---- 30 | synchronized void clean() 31 | ---- 32 | 33 | `clean`...FIXME 34 | 35 | NOTE: `clean` is used when...FIXME 36 | 37 | === [[cleanRemovedTasks]] `cleanRemovedTasks` Method 38 | 39 | [source, java] 40 | ---- 41 | void cleanRemovedTasks(final long cleanupDelayMs) // <1> 42 | 43 | // private 44 | private void cleanRemovedTasks( 45 | final long cleanupDelayMs, 46 | final boolean manualUserCall) throws Exception 47 | ---- 48 | <1> Turns the `manualUserCall` flag off 49 | 50 | `cleanRemovedTasks`...FIXME 51 | 52 | NOTE: `cleanRemovedTasks` is used when...FIXME 53 | 54 | === [[creating-instance]] Creating StateDirectory Instance 55 | 56 | `StateDirectory` takes the following when created: 57 | 58 | * [[config]] link:kafka-streams-StreamsConfig.adoc[StreamsConfig] 59 | * [[time]] `Time` 60 | 61 | `StateDirectory` initializes the <>. 62 | 63 | === [[listTaskDirectories]] `listTaskDirectories` Method 64 | 65 | [source, java] 66 | ---- 67 | File[] listTaskDirectories() 68 | ---- 69 | 70 | `listTaskDirectories`...FIXME 71 | 72 | NOTE: `listTaskDirectories` is used when...FIXME 73 | 74 | === [[lock]] Locking State Directory For Task -- `lock` Method 75 | 76 | [source, java] 77 | ---- 78 | boolean lock(TaskId taskId) 79 | ---- 80 | 81 | `lock`...FIXME 82 | 83 | NOTE: `lock` is used when...FIXME 84 | -------------------------------------------------------------------------------- /kafka-streams-internals-StateListener.adoc: -------------------------------------------------------------------------------- 1 | == [[StateListener]] StateListener -- KafkaStreams State Listener 2 | 3 | `StateListener` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-StateRestoreCallbackAdapter.adoc: -------------------------------------------------------------------------------- 1 | == [[StateRestoreCallbackAdapter]] StateRestoreCallbackAdapter -- Converting StateRestoreCallbacks To RecordBatchingStateRestoreCallbacks 2 | 3 | `StateRestoreCallbackAdapter` is a factory object (_adapter_) of <>. 4 | 5 | === [[adapt]] Creating RecordBatchingStateRestoreCallback For StateRestoreCallback -- `adapt` Factory Method 6 | 7 | [source, java] 8 | ---- 9 | RecordBatchingStateRestoreCallback adapt( 10 | StateRestoreCallback restoreCallback) 11 | ---- 12 | 13 | `adapt` simply creates a <> for a <>. 14 | 15 | Internally, `adapt`...FIXME 16 | 17 | [NOTE] 18 | ==== 19 | `adapt` is used when: 20 | 21 | * <> is created 22 | 23 | * `GlobalStateManagerImpl` is requested to <> 24 | 25 | * `ProcessorStateManager` is requested to <> 26 | ==== 27 | -------------------------------------------------------------------------------- /kafka-streams-internals-StateStoreNode.adoc: -------------------------------------------------------------------------------- 1 | == [[StateStoreNode]] StateStoreNode 2 | 3 | `StateStoreNode` is a concrete <> that is <> exclusively when `InternalStreamsBuilder` is requested to <>. 4 | 5 | In other words, `StateStoreNode` represents <> operator. 6 | 7 | [[creating-instance]][[storeBuilder]] 8 | `StateStoreNode` takes a single <> to be created. 9 | 10 | === [[writeToTopology]] `writeToTopology` Method 11 | 12 | [source, java] 13 | ---- 14 | void writeToTopology( 15 | InternalTopologyBuilder topologyBuilder) 16 | ---- 17 | 18 | NOTE: `writeToTopology` is part of the <> to...FIXME. 19 | 20 | `writeToTopology` simply requests the given <> to <> (as the <>). 21 | -------------------------------------------------------------------------------- /kafka-streams-internals-StateStoreProvider.adoc: -------------------------------------------------------------------------------- 1 | == [[StateStoreProvider]] StateStoreProvider 2 | 3 | `StateStoreProvider` is the <> of <> that <>. 4 | 5 | [[contract]] 6 | [source, java] 7 | ---- 8 | package org.apache.kafka.streams.state.internals; 9 | 10 | interface StateStoreProvider { 11 | List stores(String storeName, QueryableStoreType queryableStoreType); 12 | } 13 | ---- 14 | 15 | .StateStoreProvider Contract 16 | [cols="1,2",options="header",width="100%"] 17 | |=== 18 | | Method 19 | | Description 20 | 21 | | `stores` 22 | | [[stores]] Used when...FIXME 23 | |=== 24 | 25 | [[implementations]] 26 | .StateStoreProviders 27 | [cols="1,2",options="header",width="100%"] 28 | |=== 29 | | StateStoreProvider 30 | | Description 31 | 32 | | link:kafka-streams-internals-GlobalStateStoreProvider.adoc[GlobalStateStoreProvider] 33 | | [[GlobalStateStoreProvider]] 34 | 35 | | link:kafka-streams-internals-StreamThreadStateStoreProvider.adoc[StreamThreadStateStoreProvider] 36 | | [[StreamThreadStateStoreProvider]] 37 | 38 | | link:kafka-streams-internals-WrappingStoreProvider.adoc[WrappingStoreProvider] 39 | | [[WrappingStoreProvider]] 40 | |=== 41 | -------------------------------------------------------------------------------- /kafka-streams-internals-StaticTopicNameExtractor.adoc: -------------------------------------------------------------------------------- 1 | == [[StaticTopicNameExtractor]] StaticTopicNameExtractor 2 | 3 | `StaticTopicNameExtractor` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-StickyTaskAssignor.adoc: -------------------------------------------------------------------------------- 1 | == [[StickyTaskAssignor]] StickyTaskAssignor 2 | 3 | `StickyTaskAssignor` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-StreamSinkNode.adoc: -------------------------------------------------------------------------------- 1 | == [[StreamSinkNode]] StreamSinkNode 2 | 3 | `StreamSinkNode` is a <> that is <> exclusively when `KStreamImpl` is requested to <>. 4 | 5 | In other words, `StreamSinkNode` represents <> operator. 6 | 7 | [[creating-instance]] 8 | `StreamSinkNode` takes the following when created: 9 | 10 | * [[nodeName]] *Name* of the node 11 | * [[topicNameExtractor]] <> 12 | * [[producedInternal]] <> 13 | 14 | === [[writeToTopology]] `writeToTopology` Method 15 | 16 | [source, java] 17 | ---- 18 | void writeToTopology(final InternalTopologyBuilder topologyBuilder) 19 | ---- 20 | 21 | NOTE: `writeToTopology` is part of the <> to...FIXME. 22 | 23 | `writeToTopology` simply requests the given `InternalTopologyBuilder` to <> (per <>). 24 | 25 | `writeToTopology` creates a new <> as the <> when no <> is specified and the <> is a <>. 26 | -------------------------------------------------------------------------------- /kafka-streams-internals-StreamSourceNode.adoc: -------------------------------------------------------------------------------- 1 | == [[StreamSourceNode]] StreamSourceNode 2 | 3 | `StreamSourceNode` is a <> that is <> when `InternalStreamsBuilder` is requested to <>. 4 | 5 | In other words, `StreamSourceNode` represents <> and <> operators. 6 | 7 | [[creating-instance]] 8 | `StreamSourceNode` takes the following when created: 9 | 10 | * [[nodeName]] *Name* of the node 11 | * [[topicNames]] *Topic names* (`Collection`) or [[topicPattern]] *topic pattern* (Java's https://docs.oracle.com/en/java/javase/11/docs/api/java.base/java/util/regex/Pattern.html[java.util.regex.Pattern]) 12 | * [[consumedInternal]] <> 13 | 14 | === [[writeToTopology]] `writeToTopology` Method 15 | 16 | [source, java] 17 | ---- 18 | void writeToTopology(final InternalTopologyBuilder topologyBuilder) 19 | ---- 20 | 21 | NOTE: `writeToTopology` is part of the <> to...FIXME. 22 | 23 | `writeToTopology` simply requests the given `InternalTopologyBuilder` to <> (per <> and <> or <>). 24 | -------------------------------------------------------------------------------- /kafka-streams-internals-StreamTableJoinNode.adoc: -------------------------------------------------------------------------------- 1 | == [[StreamTableJoinNode]] StreamTableJoinNode 2 | 3 | `StreamTableJoinNode` is a concrete <> that represents (and is <> for) <> and <> operators. 4 | 5 | === [[creating-instance]] Creating StreamTableJoinNode Instance 6 | 7 | `StreamTableJoinNode` takes the following to be created: 8 | 9 | * [[nodeName]] Node name 10 | * [[processorParameters]] `ProcessorParameters` 11 | * [[storeNames]] Names of the state stores 12 | * [[otherJoinSideNodeName]] Node name of the other join side 13 | 14 | === [[writeToTopology]] `writeToTopology` Method 15 | 16 | [source, java] 17 | ---- 18 | void writeToTopology( 19 | InternalTopologyBuilder topologyBuilder) 20 | ---- 21 | 22 | NOTE: `writeToTopology` is part of the <> to...FIXME. 23 | 24 | `writeToTopology`...FIXME 25 | -------------------------------------------------------------------------------- /kafka-streams-internals-StreamThreadStateStoreProvider.adoc: -------------------------------------------------------------------------------- 1 | == [[StreamThreadStateStoreProvider]] StreamThreadStateStoreProvider 2 | 3 | `StreamThreadStateStoreProvider` is...FIXME 4 | 5 | === [[stores]] `stores` Method 6 | 7 | [source, java] 8 | ---- 9 | List stores( 10 | final String storeName, 11 | final QueryableStoreType queryableStoreType) 12 | ---- 13 | 14 | NOTE: `stores` is part of <> to...FIXME. 15 | 16 | `stores`...FIXME 17 | -------------------------------------------------------------------------------- /kafka-streams-internals-StreamsMetricsImpl.adoc: -------------------------------------------------------------------------------- 1 | == [[StreamsMetricsImpl]] StreamsMetricsImpl 2 | 3 | `StreamsMetricsImpl` is a concrete <> that...FIXME 4 | 5 | `StreamsMetricsImpl` is <> when: 6 | 7 | * `GlobalStreamThread` is <> 8 | 9 | * `StreamsMetricsThreadImpl` is <> 10 | 11 | When requested for the <>, `StreamsMetricsImpl` simply requests the Kafka <> for them. 12 | 13 | NOTE: link:kafka-streams-internals-StreamsMetricsThreadImpl.adoc[StreamsMetricsThreadImpl] is a custom `StreamsMetricsImpl` that is used...FIXME...for efficiency. 14 | 15 | === [[measureLatencyNs]] `measureLatencyNs` Method 16 | 17 | [source, java] 18 | ---- 19 | void measureLatencyNs(final Time time, final Runnable action, final Sensor sensor) 20 | ---- 21 | 22 | `measureLatencyNs`...FIXME 23 | 24 | NOTE: `measureLatencyNs` is used when...FIXME 25 | 26 | === [[recordLatency]] Recording Latency with Sensor -- `recordLatency` Method 27 | 28 | [source, java] 29 | ---- 30 | void recordLatency(Sensor sensor, long startNs, long endNs) 31 | ---- 32 | 33 | NOTE: `recordLatency` is part of link:kafka-streams-StreamsMetrics.adoc#recordLatency[StreamsMetrics Contract] to record a latency with a Kafka `Sensor`. 34 | 35 | `recordLatency` simply requests the input Kafka `Sensor` to record the latency (i.e. the difference between the input `endNs` and `startNs` timestamps). 36 | 37 | === [[creating-instance]] Creating StreamsMetricsImpl Instance 38 | 39 | `StreamsMetricsImpl` takes the following when created: 40 | 41 | * [[metrics]] Kafka `Metrics` 42 | * [[threadName]] Thread name 43 | 44 | `StreamsMetricsImpl` initializes the <>. 45 | -------------------------------------------------------------------------------- /kafka-streams-internals-StreamsMetricsThreadImpl.adoc: -------------------------------------------------------------------------------- 1 | == [[StreamsMetricsThreadImpl]] StreamsMetricsThreadImpl 2 | 3 | `StreamsMetricsThreadImpl` is a concrete <>. 4 | 5 | `StreamsMetricsThreadImpl` is <> for a link:kafka-streams-internals-StreamThread.adoc#create[StreamThread] (when `KafkaStreams` is link:kafka-streams-KafkaStreams.adoc#creating-instance[created]). 6 | 7 | [[internal-registries]] 8 | [[sensors]] 9 | .StreamsMetricsThreadImpl's Sensors 10 | [cols="1m,2",options="header",width="100%"] 11 | |=== 12 | | Name 13 | | Description 14 | 15 | | commitTimeSensor 16 | | [[commitTimeSensor]] 17 | 18 | | pollTimeSensor 19 | | [[pollTimeSensor]] 20 | 21 | | processTimeSensor 22 | | [[processTimeSensor]] 23 | 24 | | punctuateTimeSensor 25 | | [[punctuateTimeSensor]] 26 | 27 | | taskCreatedSensor 28 | | [[taskCreatedSensor]] 29 | 30 | | tasksClosedSensor 31 | | [[tasksClosedSensor]] 32 | 33 | | skippedRecordsSensor 34 | | [[skippedRecordsSensor]] 35 | 36 | |=== 37 | 38 | === [[creating-instance]] Creating StreamsMetricsThreadImpl Instance 39 | 40 | `StreamsMetricsThreadImpl` takes the following when created: 41 | 42 | * [[metrics]] `Metrics` 43 | * [[groupName]] `groupName` 44 | * [[prefix]] `prefix` 45 | * [[tags]] Tags (as `Map`) 46 | 47 | `StreamsMetricsThreadImpl` initializes the <>. 48 | 49 | === [[createMeter]] `createMeter` Internal Method 50 | 51 | [source, java] 52 | ---- 53 | Meter createMeter(Metrics metrics, SampledStat stat, String baseName, String descriptiveName) 54 | ---- 55 | 56 | `createMeter`...FIXME 57 | 58 | NOTE: `createMeter` is used when...FIXME 59 | 60 | === [[removeAllSensors]] `removeAllSensors` Method 61 | 62 | [source, java] 63 | ---- 64 | void removeAllSensors() 65 | ---- 66 | 67 | `removeAllSensors`...FIXME 68 | 69 | NOTE: `removeAllSensors` is used when...FIXME 70 | -------------------------------------------------------------------------------- /kafka-streams-internals-TaskAssignor.adoc: -------------------------------------------------------------------------------- 1 | == [[TaskAssignor]] TaskAssignor Contract 2 | 3 | `TaskAssignor` is the <> of <> that can <>. 4 | 5 | [[contract]] 6 | .TaskAssignor Contract 7 | [cols="1m,2",options="header",width="100%"] 8 | |=== 9 | | Method 10 | | Description 11 | 12 | | assign 13 | a| [[assign]] 14 | 15 | [source, java] 16 | ---- 17 | void assign(int numStandbyReplicas) 18 | ---- 19 | 20 | Assigns tasks to clients with a given number of standby replicas 21 | 22 | Used exclusively when `StreamsPartitionAssignor` is requested to <> 23 | |=== 24 | 25 | [[implementations]] 26 | NOTE: <> is the one and only known implementation of the <>. 27 | -------------------------------------------------------------------------------- /kafka-streams-internals-TimeOrderedKeyValueBuffer.adoc: -------------------------------------------------------------------------------- 1 | == [[TimeOrderedKeyValueBuffer]] TimeOrderedKeyValueBuffer 2 | 3 | `TimeOrderedKeyValueBuffer` is the <> of the <> for <> that <>. 4 | 5 | [[contract]] 6 | .TimeOrderedKeyValueBuffer Contract 7 | [cols="30m,70",options="header",width="100%"] 8 | |=== 9 | | Method 10 | | Description 11 | 12 | | bufferSize 13 | a| [[bufferSize]] 14 | 15 | [source, java] 16 | ---- 17 | long bufferSize() 18 | ---- 19 | 20 | Used when...FIXME 21 | 22 | | evictWhile 23 | a| [[evictWhile]] 24 | 25 | [source, java] 26 | ---- 27 | void evictWhile( 28 | Supplier predicate, 29 | Consumer> callback) 30 | ---- 31 | 32 | Used when...FIXME 33 | 34 | | minTimestamp 35 | a| [[minTimestamp]] 36 | 37 | [source, java] 38 | ---- 39 | long minTimestamp() 40 | ---- 41 | 42 | Used when...FIXME 43 | 44 | | numRecords 45 | a| [[numRecords]] 46 | 47 | [source, java] 48 | ---- 49 | int numRecords() 50 | ---- 51 | 52 | Used when...FIXME 53 | 54 | | priorValueForBuffered 55 | a| [[priorValueForBuffered]] 56 | 57 | [source, java] 58 | ---- 59 | Maybe> priorValueForBuffered(K key) 60 | ---- 61 | 62 | Used when...FIXME 63 | 64 | | put 65 | a| [[put]] 66 | 67 | [source, java] 68 | ---- 69 | void put( 70 | long time, 71 | K key, 72 | Change value, 73 | ProcessorRecordContext recordContext) 74 | ---- 75 | 76 | Used when...FIXME 77 | 78 | | setSerdesIfNull 79 | a| [[setSerdesIfNull]] 80 | 81 | [source, java] 82 | ---- 83 | void setSerdesIfNull( 84 | Serde keySerde, 85 | Serde valueSerde) 86 | ---- 87 | 88 | Used when...FIXME 89 | 90 | |=== 91 | 92 | [[implementations]] 93 | NOTE: <> is the default and only known implementation of the <> in Kafka Streams. 94 | -------------------------------------------------------------------------------- /kafka-streams-internals-TimeWindow.adoc: -------------------------------------------------------------------------------- 1 | == [[TimeWindow]] TimeWindow 2 | 3 | `TimeWindow` is a concrete <>. 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-TimestampedKeyValueStoreBuilder.adoc: -------------------------------------------------------------------------------- 1 | == [[TimestampedKeyValueStoreBuilder]] TimestampedKeyValueStoreBuilder 2 | 3 | `TimestampedKeyValueStoreBuilder` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-TimestampedWindowStoreBuilder.adoc: -------------------------------------------------------------------------------- 1 | == [[TimestampedWindowStoreBuilder]] TimestampedWindowStoreBuilder 2 | 3 | `TimestampedWindowStoreBuilder` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-TopicsInfo.adoc: -------------------------------------------------------------------------------- 1 | == [[TopicsInfo]] TopicsInfo 2 | 3 | [[creating-instance]] 4 | `TopicsInfo` is a simple "container" with the following: 5 | 6 | * [[sinkTopics]] Names of the sink topics 7 | * [[sourceTopics]] Names of the source topics 8 | * [[repartitionSourceTopics]] `Map` 9 | * [[stateChangelogTopics]] `Map` 10 | 11 | `TopicsInfo` is <> exclusively when `InternalTopologyBuilder` is requested for the <> (when a node group has at least one source topic, incl. repartition or state changelog topics). 12 | -------------------------------------------------------------------------------- /kafka-streams-internals-UnlimitedWindow.adoc: -------------------------------------------------------------------------------- 1 | == [[UnlimitedWindow]] UnlimitedWindow 2 | 3 | `UnlimitedWindow` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-internals-UnwindowedChangelogTopicConfig.adoc: -------------------------------------------------------------------------------- 1 | == [[UnwindowedChangelogTopicConfig]] UnwindowedChangelogTopicConfig 2 | 3 | `UnwindowedChangelogTopicConfig` is an link:kafka-streams-internals-InternalTopicConfig.adoc[InternalTopicConfig] that...FIXME 4 | 5 | `UnwindowedChangelogTopicConfig` is <> exclusively when `InternalTopologyBuilder` is requested for link:kafka-streams-internals-InternalTopologyBuilder.adoc#createChangelogTopicConfig[create an InternalTopicConfig (for a given name and StateStoreFactory)]. 6 | 7 | [[internal-registries]] 8 | [[UNWINDOWED_STORE_CHANGELOG_TOPIC_DEFAULT_OVERRIDES]] 9 | When <>, `UnwindowedChangelogTopicConfig` creates a `UNWINDOWED_STORE_CHANGELOG_TOPIC_DEFAULT_OVERRIDES` configuration properties that...FIXME 10 | 11 | === [[creating-instance]] Creating UnwindowedChangelogTopicConfig Instance 12 | 13 | `UnwindowedChangelogTopicConfig` takes the following when created: 14 | 15 | * [[name]] Name 16 | * [[topicConfigs]] Topic configs (as `Map`) 17 | 18 | `UnwindowedChangelogTopicConfig` initializes the <>. 19 | -------------------------------------------------------------------------------- /kafka-streams-internals-WindowToTimestampedWindowByteStoreAdapter.adoc: -------------------------------------------------------------------------------- 1 | == [[WindowToTimestampedWindowByteStoreAdapter]] WindowToTimestampedWindowByteStoreAdapter 2 | 3 | `WindowToTimestampedWindowByteStoreAdapter` is a concrete <> that...FIXME 4 | 5 | [[persistent]] 6 | `WindowToTimestampedWindowByteStoreAdapter` is always <>. 7 | -------------------------------------------------------------------------------- /kafka-streams-internals-WindowedChangelogTopicConfig.adoc: -------------------------------------------------------------------------------- 1 | == [[WindowedChangelogTopicConfig]] WindowedChangelogTopicConfig 2 | 3 | `WindowedChangelogTopicConfig` is an link:kafka-streams-internals-InternalTopicConfig.adoc[InternalTopicConfig] that...FIXME 4 | 5 | `WindowedChangelogTopicConfig` is <> exclusively when `InternalTopologyBuilder` is requested for link:kafka-streams-internals-InternalTopologyBuilder.adoc#createChangelogTopicConfig[create an InternalTopicConfig (for a given name and StateStoreFactory)]. 6 | 7 | [[internal-registries]] 8 | [[WINDOWED_STORE_CHANGELOG_TOPIC_DEFAULT_OVERRIDES]] 9 | When <>, `WindowedChangelogTopicConfig` creates a `WINDOWED_STORE_CHANGELOG_TOPIC_DEFAULT_OVERRIDES` configuration properties that...FIXME 10 | 11 | === [[creating-instance]] Creating WindowedChangelogTopicConfig Instance 12 | 13 | `WindowedChangelogTopicConfig` takes the following when created: 14 | 15 | * [[name]] Name 16 | * [[topicConfigs]] Topic configs (as `Map`) 17 | 18 | `WindowedChangelogTopicConfig` initializes the <>. 19 | 20 | === [[setRetentionMs]] `setRetentionMs` Method 21 | 22 | [source, java] 23 | ---- 24 | void setRetentionMs(final long retentionMs) 25 | ---- 26 | 27 | `setRetentionMs`...FIXME 28 | 29 | NOTE: `setRetentionMs` is used when...FIXME 30 | -------------------------------------------------------------------------------- /kafka-streams-internals-WindowedSerializer.adoc: -------------------------------------------------------------------------------- 1 | == [[WindowedSerializer]] WindowedSerializer Contract 2 | 3 | `WindowedSerializer` is the <> of the https://kafka.apache.org/22/javadoc/org/apache/kafka/common/serialization/Serializer.html[Serializer] contract in Apache Kafka for <> that can <>. 4 | 5 | [[contract]] 6 | [[serializeBaseKey]] 7 | `WindowedSerializer` defines the single `serializeBaseKey` method to convert (_serialize_) objects of <> type to bytes. 8 | 9 | [source, java] 10 | ---- 11 | byte[] serializeBaseKey(String topic, Windowed data) 12 | ---- 13 | 14 | `serializeBaseKey` is used exclusively when `WindowedStreamPartitioner` is requested to <>. 15 | 16 | [[implementations]] 17 | .WindowedSerializers 18 | [cols="1m,2",options="header",width="100%"] 19 | |=== 20 | | WindowedSerializer 21 | | Description 22 | 23 | | SessionWindowedSerializer 24 | | [[SessionWindowedSerializer]] 25 | 26 | | <> 27 | | [[TimeWindowedSerializer]] 28 | 29 | |=== 30 | -------------------------------------------------------------------------------- /kafka-streams-internals-WindowedStreamPartitioner.adoc: -------------------------------------------------------------------------------- 1 | == [[WindowedStreamPartitioner]] WindowedStreamPartitioner -- Default StreamPartitioner of Windowed Keys 2 | 3 | `WindowedStreamPartitioner` is a <> of <> keys (and `V` values). 4 | 5 | `WindowedStreamPartitioner` is used as the default `StreamPartitioner` when <> is used for record keys. 6 | 7 | `WindowedStreamPartitioner` is <> exclusively when `StreamSinkNode` is requested to <>. 8 | 9 | [[creating-instance]] 10 | [[serializer]] 11 | `WindowedStreamPartitioner` takes a <> when created. 12 | 13 | [[partition]] 14 | When requested to <>, `WindowedStreamPartitioner` requests the <> to <> (for the given topic and `windowedKey`). It then generates 32-bit murmur2 hash from the byte array and chooses the partition. 15 | -------------------------------------------------------------------------------- /kafka-streams-internals-WrappingStoreProvider.adoc: -------------------------------------------------------------------------------- 1 | == [[WrappingStoreProvider]] WrappingStoreProvider 2 | 3 | `WrappingStoreProvider` is...FIXME 4 | -------------------------------------------------------------------------------- /kafka-streams-multi-instance-kafka-streams-application.adoc: -------------------------------------------------------------------------------- 1 | == Multi-Instance Kafka Streams Applications 2 | 3 | A single Kafka Streams application can be executed in a group (of individual <> that are identified by the same <>). 4 | 5 | The stream processing clients can be run on the same physical machine or separate nodes. 6 | 7 | From the perspective of a Apache Kafka cluster the instances all together act as a *consumer group* (and, by the rules of consumer group, they share the partitions in such a way that no two instances access a partition). 8 | 9 | If a topology uses local state stores, they are owned exclusively (and not shared) by the instances themselves that have an exclusive access to the stores (that hold state based on the records in the exclusive set of partitions assigned to them). 10 | 11 | In such a stream processing group, every `KafkaStreams` streams client can however expose a user-defined endpoint (as a pair of host and port using <> configuration property) that allows for <>. 12 | 13 | The KafkaStreams instances become *discoverable* as a feature of KafkaStreams libraray not some external discovery framework. 14 | 15 | That distributed yet interconnected Kafka Streams application allows for developing APIs and services that could use the state distributed across stream processing nodes (that span over multiple machines). 16 | 17 | TIP: Read up more on the feature in the initial code drop as part of https://issues.apache.org/jira/browse/KAFKA-3914[KAFKA-3914: Global discovery of state stores] and https://cwiki.apache.org/confluence/display/KAFKA/KIP-67%3A+Queryable+state+for+Kafka+Streams[KIP-67: Queryable state for Kafka Streams]. 18 | -------------------------------------------------------------------------------- /kafka-streams-processor-api.adoc: -------------------------------------------------------------------------------- 1 | == Low-Level Processor API 2 | 3 | Kafka Streams offers *Processor API* with the following low-level data processing abstractions: 4 | 5 | * <> for stream processing nodes 6 | 7 | * <> to access the topology and record metadata 8 | 9 | * <> for creating `Processor` instances 10 | 11 | * <> 12 | -------------------------------------------------------------------------------- /kafka-streams-scala-Consumed.adoc: -------------------------------------------------------------------------------- 1 | == [[Consumed]] Consumed 2 | 3 | `Consumed` Scala object is part of <> that defines <> factory methods for creating <> instances with key and value `Serde` objects available in implicit scope. 4 | 5 | [[with]] 6 | .Consumed's "with" Factory Methods 7 | [source, java] 8 | ---- 9 | // Note the backticks to use "with" reserved keyword 10 | // ConsumedJ is simply an import alias for the Java-aware Consumed 11 | 12 | `with`[K, V]( 13 | timestampExtractor: TimestampExtractor, 14 | resetPolicy: Topology.AutoOffsetReset 15 | )(implicit keySerde: Serde[K], valueSerde: Serde[V]): ConsumedJ[K, V] 16 | 17 | `with`[K, V]( 18 | implicit keySerde: Serde[K], valueSerde: Serde[V]): ConsumedJ[K, V] 19 | 20 | `with`[K, V](timestampExtractor: TimestampExtractor)( 21 | implicit keySerde: Serde[K], valueSerde: Serde[V]): ConsumedJ[K, V] 22 | 23 | `with`[K, V](resetPolicy: Topology.AutoOffsetReset)( 24 | implicit keySerde: Serde[K], valueSerde: Serde[V]): ConsumedJ[K, V] 25 | ---- 26 | 27 | .Example: Creating Consumed Instance using Scala API for Kafka Streams 28 | [source, scala] 29 | ---- 30 | import org.apache.kafka.streams.scala._ 31 | import ImplicitConversions._ 32 | import Serdes._ 33 | 34 | import org.apache.kafka.streams.scala.kstream.Consumed 35 | val consumed = Consumed.`with`[Long, String] 36 | 37 | scala> :type consumed 38 | org.apache.kafka.streams.kstream.Consumed[Long,String] 39 | ---- 40 | 41 | TIP: Read up on https://docs.scala-lang.org/tour/implicit-parameters.html[Implicit Parameters]. 42 | -------------------------------------------------------------------------------- /kafka-streams-scala-Grouped.adoc: -------------------------------------------------------------------------------- 1 | == [[Grouped]] Grouped 2 | 3 | `Grouped` Scala object is part of <> that defines <> factory methods for creating <> instances with key and value `Serdes` objects available in implicit scope. 4 | 5 | [[with]] 6 | .Grouped's "with" Factory Methods 7 | [source, java] 8 | ---- 9 | // Note the backticks to use "with" reserved keyword 10 | // GroupedJ is simply an import alias for the Java-aware Grouped 11 | 12 | `with`[K, V]( 13 | implicit keySerde: Serde[K], valueSerde: Serde[V]): GroupedJ[K, V] 14 | 15 | `with`[K, V](name: String)( 16 | implicit keySerde: Serde[K], valueSerde: Serde[V]): GroupedJ[K, V] 17 | ---- 18 | 19 | .Example: Creating Grouped Instance using Scala API for Kafka Streams 20 | [source, scala] 21 | ---- 22 | import org.apache.kafka.streams.scala._ 23 | import ImplicitConversions._ 24 | import Serdes._ 25 | 26 | import org.apache.kafka.streams.scala.kstream.Grouped 27 | val grouped = Grouped.`with`[Long, String] 28 | 29 | scala> :type grouped 30 | org.apache.kafka.streams.kstream.Grouped[Long,String] 31 | ---- 32 | 33 | TIP: Read up on https://docs.scala-lang.org/tour/implicit-parameters.html[Implicit Parameters]. 34 | -------------------------------------------------------------------------------- /kafka-streams-scala-Materialized.adoc: -------------------------------------------------------------------------------- 1 | == [[Materialized]] Materialized 2 | 3 | `Materialized` Scala object is part of <> that defines <> and <> factory methods for creating <> instances with key and value `Serde` objects available in implicit scope. 4 | 5 | [[with]] 6 | [[as]] 7 | .Materialized's Factory Method 8 | [source, java] 9 | ---- 10 | // Note the backticks to use "with" reserved keyword 11 | // MaterializedJ is simply an import alias for the Java-aware Materialized 12 | // ByteArray*Store types are type aliases for *Store[Bytes, Array[Byte]] 13 | // e.g. ByteArrayWindowStore = WindowStore[Bytes, Array[Byte]] 14 | 15 | `with`[K, V, S <: StateStore]( 16 | implicit keySerde: Serde[K], valueSerde: Serde[V]): MaterializedJ[K, V, S] 17 | 18 | as[K, V, S <: StateStore](storeName: String)( 19 | implicit keySerde: Serde[K], valueSerde: Serde[V]): MaterializedJ[K, V, S] 20 | 21 | as[K, V](supplier: KeyValueBytesStoreSupplier)( 22 | implicit keySerde: Serde[K], valueSerde: Serde[V]): MaterializedJ[K, V, ByteArrayKeyValueStore] 23 | 24 | as[K, V](supplier: SessionBytesStoreSupplier)( 25 | implicit keySerde: Serde[K], valueSerde: Serde[V]): MaterializedJ[K, V, ByteArraySessionStore] 26 | 27 | as[K, V](supplier: WindowBytesStoreSupplier)( 28 | implicit keySerde: Serde[K], valueSerde: Serde[V]): MaterializedJ[K, V, ByteArrayWindowStore] 29 | ---- 30 | 31 | .Example: Creating Materialized Instance using Scala API for Kafka Streams 32 | [source, scala] 33 | ---- 34 | import org.apache.kafka.streams.scala._ 35 | import ImplicitConversions._ 36 | import Serdes._ 37 | 38 | import org.apache.kafka.streams.scala.kstream.Materialized 39 | import org.apache.kafka.streams.state.WindowStore 40 | val materialized = Materialized.`with`[Long, String, ByteArrayWindowStore] 41 | 42 | scala> :type materialized 43 | org.apache.kafka.streams.kstream.Materialized[Long,String,org.apache.kafka.streams.scala.ByteArrayWindowStore] 44 | ---- 45 | 46 | TIP: Read up on https://docs.scala-lang.org/tour/implicit-parameters.html[Implicit Parameters]. 47 | -------------------------------------------------------------------------------- /kafka-streams-scala-Produced.adoc: -------------------------------------------------------------------------------- 1 | == [[Produced]] Produced 2 | 3 | `Produced` Scala object is part of <> that defines <> factory methods for creating <> instances with key and value `Serde` objects available in implicit scope. 4 | 5 | [[with]] 6 | .Produced's "with" Factory Methods 7 | [source, java] 8 | ---- 9 | // Note the backticks to use "with" reserved keyword 10 | // ProducedJ is simply an import alias for the Java-aware Produced 11 | 12 | `with`[K, V]( 13 | implicit keySerde: Serde[K], valueSerde: Serde[V]): ProducedJ[K, V] 14 | 15 | `with`[K, V](partitioner: StreamPartitioner[K, V])( 16 | implicit keySerde: Serde[K], valueSerde: Serde[V]): ProducedJ[K, V] 17 | ---- 18 | 19 | .Example: Creating Produced Instance using Scala API for Kafka Streams 20 | [source, scala] 21 | ---- 22 | import org.apache.kafka.streams.scala._ 23 | import ImplicitConversions._ 24 | import Serdes._ 25 | 26 | import org.apache.kafka.streams.scala.kstream.Produced 27 | val produced = Produced.`with`[Long, String] 28 | 29 | scala> :type produced 30 | org.apache.kafka.streams.kstream.Produced[Long,String] 31 | ---- 32 | 33 | TIP: Read up on https://docs.scala-lang.org/tour/implicit-parameters.html[Implicit Parameters]. 34 | -------------------------------------------------------------------------------- /kafka-streams-scala-Serdes.adoc: -------------------------------------------------------------------------------- 1 | == [[Serdes]] Serdes 2 | 3 | `Serdes` is a Scala object that contains the <> with serializers and deserializers for known Scala and Java types and <> to create a new `Serde[T]` (from `T` or `(String, T)`). 4 | 5 | You should import `Serdes` to use the implicit conversions in a Scala application that uses the Kafka Streams library. 6 | 7 | [source, scala] 8 | ---- 9 | import org.apache.kafka.streams.scala.Serdes._ 10 | ---- 11 | 12 | You should also use `Serdes` when defining `StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG` and `StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG`. 13 | 14 | [source, scala] 15 | ---- 16 | import org.apache.kafka.streams.scala.Serdes._ 17 | Serdes.String.getClass.getName 18 | ---- 19 | 20 | [[implicits]] 21 | .Implicit Conversions 22 | [cols="1m,2",options="header",width="100%"] 23 | |=== 24 | | Name 25 | | Serde 26 | 27 | | ByteArray 28 | | [[ByteArray]] Serde[Array[Byte]] 29 | 30 | | Bytes 31 | | [[Bytes]] Serde[org.apache.kafka.common.utils.Bytes] 32 | 33 | | Double 34 | | [[Double]] Serde[Double] 35 | 36 | | Float 37 | | [[Float]] Serde[Float] 38 | 39 | | Integer 40 | | [[Integer]] Serde[Int] 41 | 42 | | JavaDouble 43 | | [[JavaDouble]] Serde[java.lang.Double] 44 | 45 | | JavaFloat 46 | | [[JavaFloat]] Serde[java.lang.Float] 47 | 48 | | JavaInteger 49 | | [[JavaInteger]] Serde[java.lang.Integer] 50 | 51 | | JavaLong 52 | | [[JavaLong]] Serde[java.lang.Long] 53 | 54 | | Long 55 | | [[Long]] Serde[Long] 56 | 57 | | String 58 | | [[String]] Serde[String] 59 | 60 | | sessionWindowedSerde[T] 61 | | [[sessionWindowedSerde]] <> 62 | 63 | | timeWindowedSerde[T] 64 | | [[timeWindowedSerde]] <> 65 | |=== 66 | 67 | === [[fromFn]] `fromFn` Method 68 | 69 | [source, scala] 70 | ---- 71 | fromFn[T >: Null]( 72 | serializer: T => Array[Byte], 73 | deserializer: Array[Byte] => Option[T]): Serde[T] 74 | fromFn[T >: Null]( 75 | serializer: (String, T) => Array[Byte], 76 | deserializer: (String, Array[Byte]) => Option[T]): Serde[T] 77 | ---- 78 | 79 | `fromFn`...FIXME 80 | 81 | NOTE: `fromFn` is used when...FIXME 82 | -------------------------------------------------------------------------------- /kafka-streams-scala.adoc: -------------------------------------------------------------------------------- 1 | == Scala API for Kafka Streams 2 | 3 | *Scala API for Kafka Streams* is a separate Kafka Streams module (a Scala library) that acts as a wrapper over the existing Java API for Kafka Streams. 4 | 5 | The Scala API is available in `org.apache.kafka.streams.scala` package. 6 | 7 | As a separate Scala library you have to define the dependency in `build.sbt`. 8 | 9 | [source, scala] 10 | ---- 11 | // Note two percent signs (%%) to encode Scala version 12 | libraryDependencies += "org.apache.kafka" %% "kafka-streams-scala" % "{{ book.kafka_version }}" 13 | ---- 14 | 15 | [[types]] 16 | The Scala API for Kafka Streams defines Scala-friendly types that wrap the corresponding Kafka Streams types and simply delegate all method calls to the underlying Java object with the purpose of making it much more expressive, with less boilerplate and more succinct. 17 | 18 | * <> 19 | * <> 20 | * <> 21 | * <> 22 | * <> 23 | * <> 24 | * <> 25 | 26 | Beside the Scala-friendly types, the Scala API for Kafka Streams defines implicit conversions, i.e. <>, and <>. 27 | 28 | [source, scala] 29 | ---- 30 | import org.apache.kafka.streams.scala._ 31 | import ImplicitConversions._ 32 | import Serdes._ 33 | ---- 34 | 35 | The Scala API for Kafka Streams comes with <> Scala object that allows for creating <> instances with key and value `Serdes` objects available in implicit scope. 36 | 37 | You could also use `Serdes` when defining `StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG` and `StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG`. 38 | 39 | [source, scala] 40 | ---- 41 | import org.apache.kafka.streams.scala.Serdes._ 42 | Serdes.String.getClass.getName 43 | ---- 44 | 45 | NOTE: Scala API for Kafka Streams was added in Kafka Streams 2.0.0 as link:++https://issues.apache.org/jira/browse/KAFKA-6670++[KAFKA-6670 Implement a Scala wrapper library for Kafka Streams]. 46 | -------------------------------------------------------------------------------- /kafka-streams-stateful-stream-processing.adoc: -------------------------------------------------------------------------------- 1 | == Stateful Stream Processing 2 | 3 | *Stateful Stream Processing* is...FIXME 4 | 5 | NOTE: A `StateStore` can be *local* or *global* (although it is a property of a <>). 6 | 7 | Kafka Streams developers use <> utility for creating state stores. 8 | 9 | NOTE: A `StateStore` can be backed by a changelog topic in the Kafka cluster for fault-tolerance, i.e. with logging enabled (although it is an implementation detail, and not part of the `StateStore` contract). Kafka Streams developers use <> or <> APIs to control logging. 10 | 11 | NOTE: A `StateStore` can be cached for performance, i.e. with caching enabled (although it is an implementation detail, and not part of the `StateStore` contract). Kafka Streams developers use <> or <> APIs to control logging. 12 | -------------------------------------------------------------------------------- /kafka-streams-streams-dsl.adoc: -------------------------------------------------------------------------------- 1 | == Streams DSL -- High-Level Stream Processing DSL 2 | 3 | Kafka Streams offers *Streams DSL* with high-level data stream abstractions for streams, tables, state stores, topologies. 4 | 5 | Streams DSL includes: 6 | 7 | * <> for defining a topology 8 | 9 | * <> for working with record streams 10 | 11 | * <> for working with changelog streams 12 | 13 | * <> for working with global changelog streams 14 | 15 | Streams DSL also includes the following for streaming aggregations and joins: 16 | 17 | * <> 18 | 19 | * <> 20 | 21 | Streams DSL is designed for Java developers primarily, but also offers <> for Scala developers. 22 | -------------------------------------------------------------------------------- /kafka-streams-why.adoc: -------------------------------------------------------------------------------- 1 | == Why Kafka Streams 2 | 3 | In this section you can find reasons that could make Kafka Streams a viable contender among the solutions for your stream processing project. 4 | 5 | 1. <> 6 | 1. <> 7 | 8 | === [[KafkaAsDataStorage]] Apache Kafka as the Data Storage 9 | 10 | Kafka Streams is a library for developing applications for record stream processing where the data is in topics in Apache Kafka. 11 | 12 | NOTE: Kafka Streams supports consuming from and producing records to a single Apache Kafka cluster only (unless you use `foreach` operator for producing records to another Kafka cluster which means that you are in charge of managing a Kafka Producer to that cluster). 13 | 14 | === [[ConsumerRebalanceListener]] Writing Kafka ConsumerRebalanceListener Is Tricky 15 | 16 | Quoting the documentation of https://kafka.apache.org/22/javadoc/org/apache/kafka/clients/consumer/ConsumerRebalanceListener.html[org.apache.kafka.clients.consumer.ConsumerRebalanceListener]: 17 | 18 | > *ConsumerRebalanceListener* is a callback interface that the user can implement to trigger custom actions when the set of partitions assigned to the consumer changes. 19 | 20 | > When Kafka is managing the group membership, a partition re-assignment will be triggered any time the members of the group change or the subscription of the members changes. This can occur when processes die, new process instances are added or old instances come back to life after failure. Rebalances can also be triggered by changes affecting the subscribed topics (e.g. when the number of partitions is administratively adjusted). 21 | --------------------------------------------------------------------------------