public class Topology extends Object
ProcessorTopology
.
A topology is an acyclic graph of sources, processors, and sinks.
A source
is a node in the graph that consumes one or more Kafka topics and forwards them to its
successor nodes.
A processor
is a node in the graph that receives input records from upstream nodes, processes the
records, and optionally forwarding new records to one or all of its downstream nodes.
Finally, a sink
is a node in the graph that receives records from upstream nodes and writes them to
a Kafka topic.
A Topology
allows you to construct an acyclic graph of these nodes, and then passed into a new
KafkaStreams
instance that will then begin consuming, processing, and producing
records
.Modifier and Type | Class and Description |
---|---|
static class |
Topology.AutoOffsetReset
Sets the
auto.offset.reset configuration when
adding a source processor or when creating KStream
or KTable via StreamsBuilder . |
Constructor and Description |
---|
Topology() |
Modifier and Type | Method and Description |
---|---|
<K,V> Topology |
addGlobalStore(StoreBuilder<?> storeBuilder,
String sourceName,
Deserializer<K> keyDeserializer,
Deserializer<V> valueDeserializer,
String topic,
String processorName,
ProcessorSupplier<K,V> stateUpdateSupplier)
Deprecated.
Since 2.7.0. Use
addGlobalStore(StoreBuilder, String, Deserializer, Deserializer, String, String, ProcessorSupplier) instead. |
<KIn,VIn> Topology |
addGlobalStore(StoreBuilder<?> storeBuilder,
String sourceName,
Deserializer<KIn> keyDeserializer,
Deserializer<VIn> valueDeserializer,
String topic,
String processorName,
ProcessorSupplier<KIn,VIn,Void,Void> stateUpdateSupplier)
Adds a global
StateStore to the topology. |
<K,V> Topology |
addGlobalStore(StoreBuilder<?> storeBuilder,
String sourceName,
TimestampExtractor timestampExtractor,
Deserializer<K> keyDeserializer,
Deserializer<V> valueDeserializer,
String topic,
String processorName,
ProcessorSupplier<K,V> stateUpdateSupplier)
Deprecated.
|
<KIn,VIn> Topology |
addGlobalStore(StoreBuilder<?> storeBuilder,
String sourceName,
TimestampExtractor timestampExtractor,
Deserializer<KIn> keyDeserializer,
Deserializer<VIn> valueDeserializer,
String topic,
String processorName,
ProcessorSupplier<KIn,VIn,Void,Void> stateUpdateSupplier)
Adds a global
StateStore to the topology. |
<KIn,VIn,KOut,VOut> |
addProcessor(String name,
ProcessorSupplier<KIn,VIn,KOut,VOut> supplier,
String... parentNames)
Add a new processor node that receives and processes records output by one or more parent source or processor
node.
|
Topology |
addProcessor(String name,
ProcessorSupplier supplier,
String... parentNames)
Deprecated.
Since 2.7.0 Use
addProcessor(String, ProcessorSupplier, String...) instead. |
<K,V> Topology |
addSink(String name,
String topic,
Serializer<K> keySerializer,
Serializer<V> valueSerializer,
StreamPartitioner<? super K,? super V> partitioner,
String... parentNames)
Add a new sink that forwards records from upstream parent processor and/or source nodes to the named Kafka topic.
|
<K,V> Topology |
addSink(String name,
String topic,
Serializer<K> keySerializer,
Serializer<V> valueSerializer,
String... parentNames)
Add a new sink that forwards records from upstream parent processor and/or source nodes to the named Kafka topic.
|
<K,V> Topology |
addSink(String name,
String topic,
StreamPartitioner<? super K,? super V> partitioner,
String... parentNames)
Add a new sink that forwards records from upstream parent processor and/or source nodes to the named Kafka topic,
using the supplied partitioner.
|
Topology |
addSink(String name,
String topic,
String... parentNames)
Add a new sink that forwards records from upstream parent processor and/or source nodes to the named Kafka topic.
|
<K,V> Topology |
addSink(String name,
TopicNameExtractor<K,V> topicExtractor,
Serializer<K> keySerializer,
Serializer<V> valueSerializer,
StreamPartitioner<? super K,? super V> partitioner,
String... parentNames)
Add a new sink that forwards records from upstream parent processor and/or source nodes to Kafka topics based on
topicExtractor . |
<K,V> Topology |
addSink(String name,
TopicNameExtractor<K,V> topicExtractor,
Serializer<K> keySerializer,
Serializer<V> valueSerializer,
String... parentNames)
Add a new sink that forwards records from upstream parent processor and/or source nodes to Kafka topics based on
topicExtractor . |
<K,V> Topology |
addSink(String name,
TopicNameExtractor<K,V> topicExtractor,
StreamPartitioner<? super K,? super V> partitioner,
String... parentNames)
Add a new sink that forwards records from upstream parent processor and/or source nodes to Kafka topics based on
topicExtractor ,
using the supplied partitioner. |
<K,V> Topology |
addSink(String name,
TopicNameExtractor<K,V> topicExtractor,
String... parentNames)
Add a new sink that forwards records from upstream parent processor and/or source nodes to Kafka topics based on
topicExtractor . |
Topology |
addSource(String name,
Deserializer<?> keyDeserializer,
Deserializer<?> valueDeserializer,
Pattern topicPattern)
Add a new source that consumes from topics matching the given pattern and forwards the records to child processor
and/or sink nodes.
|
Topology |
addSource(String name,
Deserializer<?> keyDeserializer,
Deserializer<?> valueDeserializer,
String... topics)
Add a new source that consumes the named topics and forwards the records to child processor and/or sink nodes.
|
Topology |
addSource(String name,
Pattern topicPattern)
Add a new source that consumes from topics matching the given pattern
and forward the records to child processor and/or sink nodes.
|
Topology |
addSource(String name,
String... topics)
Add a new source that consumes the named topics and forward the records to child processor and/or sink nodes.
|
Topology |
addSource(TimestampExtractor timestampExtractor,
String name,
Pattern topicPattern)
Add a new source that consumes from topics matching the given pattern
and forward the records to child processor and/or sink nodes.
|
Topology |
addSource(TimestampExtractor timestampExtractor,
String name,
String... topics)
Add a new source that consumes the named topics and forward the records to child processor and/or sink nodes.
|
Topology |
addSource(Topology.AutoOffsetReset offsetReset,
String name,
Deserializer<?> keyDeserializer,
Deserializer<?> valueDeserializer,
Pattern topicPattern)
Add a new source that consumes from topics matching the given pattern and forwards the records to child processor
and/or sink nodes.
|
Topology |
addSource(Topology.AutoOffsetReset offsetReset,
String name,
Deserializer<?> keyDeserializer,
Deserializer<?> valueDeserializer,
String... topics)
Add a new source that consumes from topics matching the given pattern and forwards the records to child processor
and/or sink nodes.
|
Topology |
addSource(Topology.AutoOffsetReset offsetReset,
String name,
Pattern topicPattern)
Add a new source that consumes from topics matching the given pattern
and forward the records to child processor and/or sink nodes.
|
Topology |
addSource(Topology.AutoOffsetReset offsetReset,
String name,
String... topics)
Add a new source that consumes the named topics and forward the records to child processor and/or sink nodes.
|
Topology |
addSource(Topology.AutoOffsetReset offsetReset,
String name,
TimestampExtractor timestampExtractor,
Deserializer<?> keyDeserializer,
Deserializer<?> valueDeserializer,
Pattern topicPattern)
Add a new source that consumes from topics matching the given pattern and forwards the records to child processor
and/or sink nodes.
|
Topology |
addSource(Topology.AutoOffsetReset offsetReset,
String name,
TimestampExtractor timestampExtractor,
Deserializer<?> keyDeserializer,
Deserializer<?> valueDeserializer,
String... topics)
Add a new source that consumes the named topics and forwards the records to child processor and/or sink nodes.
|
Topology |
addSource(Topology.AutoOffsetReset offsetReset,
TimestampExtractor timestampExtractor,
String name,
Pattern topicPattern)
Add a new source that consumes from topics matching the given pattern and forward the records to child processor
and/or sink nodes.
|
Topology |
addSource(Topology.AutoOffsetReset offsetReset,
TimestampExtractor timestampExtractor,
String name,
String... topics)
Add a new source that consumes the named topics and forward the records to child processor and/or sink nodes.
|
Topology |
addStateStore(StoreBuilder<?> storeBuilder,
String... processorNames)
Adds a state store.
|
Topology |
connectProcessorAndStateStores(String processorName,
String... stateStoreNames)
Connects the processor and the state stores.
|
TopologyDescription |
describe()
Returns a description of the specified
Topology . |
public Topology addSource(String name, String... topics)
default key deserializer
and
default value deserializer
specified in the
stream configuration
.
The default TimestampExtractor
as specified in the config
is used.name
- the unique name of the source used to reference this node when
adding processor children
.topics
- the name of one or more Kafka topics that this source is to consumeTopologyException
- if processor is already added or if topics have already been registered by another sourcepublic Topology addSource(String name, Pattern topicPattern)
default key deserializer
and
default value deserializer
specified in the
stream configuration
.
The default TimestampExtractor
as specified in the config
is used.name
- the unique name of the source used to reference this node when
adding processor children
.topicPattern
- regular expression pattern to match Kafka topics that this source is to consumeTopologyException
- if processor is already added or if topics have already been registered by another sourcepublic Topology addSource(Topology.AutoOffsetReset offsetReset, String name, String... topics)
default key deserializer
and
default value deserializer
specified in the
stream configuration
.
The default TimestampExtractor
as specified in the config
is used.offsetReset
- the auto offset reset policy to use for this source if no committed offsets found; acceptable values earliest or latestname
- the unique name of the source used to reference this node when
adding processor children
.topics
- the name of one or more Kafka topics that this source is to consumeTopologyException
- if processor is already added or if topics have already been registered by another sourcepublic Topology addSource(Topology.AutoOffsetReset offsetReset, String name, Pattern topicPattern)
default key deserializer
and
default value deserializer
specified in the
stream configuration
.
The default TimestampExtractor
as specified in the config
is used.offsetReset
- the auto offset reset policy value for this source if no committed offsets found; acceptable values earliest or latest.name
- the unique name of the source used to reference this node when
adding processor children
.topicPattern
- regular expression pattern to match Kafka topics that this source is to consumeTopologyException
- if processor is already added or if topics have already been registered by another sourcepublic Topology addSource(TimestampExtractor timestampExtractor, String name, String... topics)
default key deserializer
and
default value deserializer
specified in the
stream configuration
.timestampExtractor
- the stateless timestamp extractor used for this source,
if not specified the default extractor defined in the configs will be usedname
- the unique name of the source used to reference this node when
adding processor children
.topics
- the name of one or more Kafka topics that this source is to consumeTopologyException
- if processor is already added or if topics have already been registered by another sourcepublic Topology addSource(TimestampExtractor timestampExtractor, String name, Pattern topicPattern)
default key deserializer
and
default value deserializer
specified in the
stream configuration
.timestampExtractor
- the stateless timestamp extractor used for this source,
if not specified the default extractor defined in the configs will be usedname
- the unique name of the source used to reference this node when
adding processor children
.topicPattern
- regular expression pattern to match Kafka topics that this source is to consumeTopologyException
- if processor is already added or if topics have already been registered by another sourcepublic Topology addSource(Topology.AutoOffsetReset offsetReset, TimestampExtractor timestampExtractor, String name, String... topics)
default key deserializer
and
default value deserializer
specified in the
stream configuration
.offsetReset
- the auto offset reset policy to use for this source if no committed offsets found;
acceptable values earliest or latesttimestampExtractor
- the stateless timestamp extractor used for this source,
if not specified the default extractor defined in the configs will be usedname
- the unique name of the source used to reference this node when
adding processor children
.topics
- the name of one or more Kafka topics that this source is to consumeTopologyException
- if processor is already added or if topics have already been registered by another sourcepublic Topology addSource(Topology.AutoOffsetReset offsetReset, TimestampExtractor timestampExtractor, String name, Pattern topicPattern)
default key deserializer
and
default value deserializer
specified in the
stream configuration
.offsetReset
- the auto offset reset policy value for this source if no committed offsets found;
acceptable values earliest or latest.timestampExtractor
- the stateless timestamp extractor used for this source,
if not specified the default extractor defined in the configs will be usedname
- the unique name of the source used to reference this node when
adding processor children
.topicPattern
- regular expression pattern to match Kafka topics that this source is to consumeTopologyException
- if processor is already added or if topics have already been registered by another sourcepublic Topology addSource(String name, Deserializer<?> keyDeserializer, Deserializer<?> valueDeserializer, String... topics)
TimestampExtractor
as specified in the config
is used.name
- the unique name of the source used to reference this node when
adding processor children
keyDeserializer
- key deserializer used to read this source, if not specified the default
key deserializer defined in the configs will be usedvalueDeserializer
- value deserializer used to read this source,
if not specified the default value deserializer defined in the configs will be usedtopics
- the name of one or more Kafka topics that this source is to consumeTopologyException
- if processor is already added or if topics have already been registered by another sourcepublic Topology addSource(String name, Deserializer<?> keyDeserializer, Deserializer<?> valueDeserializer, Pattern topicPattern)
TimestampExtractor
as specified in the config
is used.name
- the unique name of the source used to reference this node when
adding processor children
keyDeserializer
- key deserializer used to read this source, if not specified the default
key deserializer defined in the configs will be usedvalueDeserializer
- value deserializer used to read this source,
if not specified the default value deserializer defined in the configs will be usedtopicPattern
- regular expression pattern to match Kafka topics that this source is to consumeTopologyException
- if processor is already added or if topics have already been registered by namepublic Topology addSource(Topology.AutoOffsetReset offsetReset, String name, Deserializer<?> keyDeserializer, Deserializer<?> valueDeserializer, String... topics)
offsetReset
- the auto offset reset policy to use for this stream if no committed offsets found;
acceptable values are earliest or latestname
- the unique name of the source used to reference this node when
adding processor children
keyDeserializer
- key deserializer used to read this source, if not specified the default
key deserializer defined in the configs will be usedvalueDeserializer
- value deserializer used to read this source,
if not specified the default value deserializer defined in the configs will be usedtopics
- the name of one or more Kafka topics that this source is to consumeTopologyException
- if processor is already added or if topics have already been registered by namepublic Topology addSource(Topology.AutoOffsetReset offsetReset, String name, Deserializer<?> keyDeserializer, Deserializer<?> valueDeserializer, Pattern topicPattern)
offsetReset
- the auto offset reset policy to use for this stream if no committed offsets found;
acceptable values are earliest or latestname
- the unique name of the source used to reference this node when
adding processor children
keyDeserializer
- key deserializer used to read this source, if not specified the default
key deserializer defined in the configs will be usedvalueDeserializer
- value deserializer used to read this source,
if not specified the default value deserializer defined in the configs will be usedtopicPattern
- regular expression pattern to match Kafka topics that this source is to consumeTopologyException
- if processor is already added or if topics have already been registered by namepublic Topology addSource(Topology.AutoOffsetReset offsetReset, String name, TimestampExtractor timestampExtractor, Deserializer<?> keyDeserializer, Deserializer<?> valueDeserializer, String... topics)
offsetReset
- the auto offset reset policy to use for this stream if no committed offsets found;
acceptable values are earliest or latest.name
- the unique name of the source used to reference this node when
adding processor children
.timestampExtractor
- the stateless timestamp extractor used for this source,
if not specified the default extractor defined in the configs will be usedkeyDeserializer
- key deserializer used to read this source, if not specified the default
key deserializer defined in the configs will be usedvalueDeserializer
- value deserializer used to read this source,
if not specified the default value deserializer defined in the configs will be usedtopics
- the name of one or more Kafka topics that this source is to consumeTopologyException
- if processor is already added or if topics have already been registered by another sourcepublic Topology addSource(Topology.AutoOffsetReset offsetReset, String name, TimestampExtractor timestampExtractor, Deserializer<?> keyDeserializer, Deserializer<?> valueDeserializer, Pattern topicPattern)
offsetReset
- the auto offset reset policy to use for this stream if no committed offsets found;
acceptable values are earliest or latestname
- the unique name of the source used to reference this node when
adding processor children
.timestampExtractor
- the stateless timestamp extractor used for this source,
if not specified the default extractor defined in the configs will be usedkeyDeserializer
- key deserializer used to read this source, if not specified the default
key deserializer defined in the configs will be usedvalueDeserializer
- value deserializer used to read this source,
if not specified the default value deserializer defined in the configs will be usedtopicPattern
- regular expression pattern to match Kafka topics that this source is to consumeTopologyException
- if processor is already added or if topics have already been registered by namepublic Topology addSink(String name, String topic, String... parentNames)
default key serializer
and
default value serializer
specified in the
stream configuration
.name
- the unique name of the sinktopic
- the name of the Kafka topic to which this sink should write its recordsparentNames
- the name of one or more source or processor nodes whose output records this sink should consume
and write to its topicTopologyException
- if parent processor is not added yet, or if this processor's name is equal to the parent's nameaddSink(String, String, StreamPartitioner, String...)
,
addSink(String, String, Serializer, Serializer, String...)
,
addSink(String, String, Serializer, Serializer, StreamPartitioner, String...)
public <K,V> Topology addSink(String name, String topic, StreamPartitioner<? super K,? super V> partitioner, String... parentNames)
default key serializer
and
default value serializer
specified in the
stream configuration
.
The sink will also use the specified StreamPartitioner
to determine how records are distributed among
the named Kafka topic's partitions.
Such control is often useful with topologies that use state
stores
in its processors.
In most other cases, however, a partitioner needs not be specified and Kafka will automatically distribute
records among partitions using Kafka's default partitioning logic.
name
- the unique name of the sinktopic
- the name of the Kafka topic to which this sink should write its recordspartitioner
- the function that should be used to determine the partition for each record processed by the sinkparentNames
- the name of one or more source or processor nodes whose output records this sink should consume
and write to its topicTopologyException
- if parent processor is not added yet, or if this processor's name is equal to the parent's nameaddSink(String, String, String...)
,
addSink(String, String, Serializer, Serializer, String...)
,
addSink(String, String, Serializer, Serializer, StreamPartitioner, String...)
public <K,V> Topology addSink(String name, String topic, Serializer<K> keySerializer, Serializer<V> valueSerializer, String... parentNames)
name
- the unique name of the sinktopic
- the name of the Kafka topic to which this sink should write its recordskeySerializer
- the key serializer
used when consuming records; may be null if the sink
should use the default key serializer
specified in the
stream configuration
valueSerializer
- the value serializer
used when consuming records; may be null if the sink
should use the default value serializer
specified in the
stream configuration
parentNames
- the name of one or more source or processor nodes whose output records this sink should consume
and write to its topicTopologyException
- if parent processor is not added yet, or if this processor's name is equal to the parent's nameaddSink(String, String, String...)
,
addSink(String, String, StreamPartitioner, String...)
,
addSink(String, String, Serializer, Serializer, StreamPartitioner, String...)
public <K,V> Topology addSink(String name, String topic, Serializer<K> keySerializer, Serializer<V> valueSerializer, StreamPartitioner<? super K,? super V> partitioner, String... parentNames)
name
- the unique name of the sinktopic
- the name of the Kafka topic to which this sink should write its recordskeySerializer
- the key serializer
used when consuming records; may be null if the sink
should use the default key serializer
specified in the
stream configuration
valueSerializer
- the value serializer
used when consuming records; may be null if the sink
should use the default value serializer
specified in the
stream configuration
partitioner
- the function that should be used to determine the partition for each record processed by the sinkparentNames
- the name of one or more source or processor nodes whose output records this sink should consume
and write to its topicTopologyException
- if parent processor is not added yet, or if this processor's name is equal to the parent's nameaddSink(String, String, String...)
,
addSink(String, String, StreamPartitioner, String...)
,
addSink(String, String, Serializer, Serializer, String...)
public <K,V> Topology addSink(String name, TopicNameExtractor<K,V> topicExtractor, String... parentNames)
topicExtractor
.
The topics that it may ever send to should be pre-created.
The sink will use the default key serializer
and
default value serializer
specified in the
stream configuration
.name
- the unique name of the sinktopicExtractor
- the extractor to determine the name of the Kafka topic to which this sink should write for each recordparentNames
- the name of one or more source or processor nodes whose output records this sink should consume
and dynamically write to topicsTopologyException
- if parent processor is not added yet, or if this processor's name is equal to the parent's nameaddSink(String, String, StreamPartitioner, String...)
,
addSink(String, String, Serializer, Serializer, String...)
,
addSink(String, String, Serializer, Serializer, StreamPartitioner, String...)
public <K,V> Topology addSink(String name, TopicNameExtractor<K,V> topicExtractor, StreamPartitioner<? super K,? super V> partitioner, String... parentNames)
topicExtractor
,
using the supplied partitioner.
The topics that it may ever send to should be pre-created.
The sink will use the default key serializer
and
default value serializer
specified in the
stream configuration
.
The sink will also use the specified StreamPartitioner
to determine how records are distributed among
the named Kafka topic's partitions.
Such control is often useful with topologies that use state
stores
in its processors.
In most other cases, however, a partitioner needs not be specified and Kafka will automatically distribute
records among partitions using Kafka's default partitioning logic.
name
- the unique name of the sinktopicExtractor
- the extractor to determine the name of the Kafka topic to which this sink should write for each recordpartitioner
- the function that should be used to determine the partition for each record processed by the sinkparentNames
- the name of one or more source or processor nodes whose output records this sink should consume
and dynamically write to topicsTopologyException
- if parent processor is not added yet, or if this processor's name is equal to the parent's nameaddSink(String, String, String...)
,
addSink(String, String, Serializer, Serializer, String...)
,
addSink(String, String, Serializer, Serializer, StreamPartitioner, String...)
public <K,V> Topology addSink(String name, TopicNameExtractor<K,V> topicExtractor, Serializer<K> keySerializer, Serializer<V> valueSerializer, String... parentNames)
topicExtractor
.
The topics that it may ever send to should be pre-created.
The sink will use the specified key and value serializers.name
- the unique name of the sinktopicExtractor
- the extractor to determine the name of the Kafka topic to which this sink should write for each recordkeySerializer
- the key serializer
used when consuming records; may be null if the sink
should use the default key serializer
specified in the
stream configuration
valueSerializer
- the value serializer
used when consuming records; may be null if the sink
should use the default value serializer
specified in the
stream configuration
parentNames
- the name of one or more source or processor nodes whose output records this sink should consume
and dynamically write to topicsTopologyException
- if parent processor is not added yet, or if this processor's name is equal to the parent's nameaddSink(String, String, String...)
,
addSink(String, String, StreamPartitioner, String...)
,
addSink(String, String, Serializer, Serializer, StreamPartitioner, String...)
public <K,V> Topology addSink(String name, TopicNameExtractor<K,V> topicExtractor, Serializer<K> keySerializer, Serializer<V> valueSerializer, StreamPartitioner<? super K,? super V> partitioner, String... parentNames)
topicExtractor
.
The topics that it may ever send to should be pre-created.
The sink will use the specified key and value serializers, and the supplied partitioner.name
- the unique name of the sinktopicExtractor
- the extractor to determine the name of the Kafka topic to which this sink should write for each recordkeySerializer
- the key serializer
used when consuming records; may be null if the sink
should use the default key serializer
specified in the
stream configuration
valueSerializer
- the value serializer
used when consuming records; may be null if the sink
should use the default value serializer
specified in the
stream configuration
partitioner
- the function that should be used to determine the partition for each record processed by the sinkparentNames
- the name of one or more source or processor nodes whose output records this sink should consume
and dynamically write to topicsTopologyException
- if parent processor is not added yet, or if this processor's name is equal to the parent's nameaddSink(String, String, String...)
,
addSink(String, String, StreamPartitioner, String...)
,
addSink(String, String, Serializer, Serializer, String...)
@Deprecated public Topology addProcessor(String name, ProcessorSupplier supplier, String... parentNames)
addProcessor(String, ProcessorSupplier, String...)
instead.supplier
provides stores via ConnectedStoreProvider.stores()
, the provided StoreBuilder
s
will be added to the topology and connected to this processor automatically.name
- the unique name of the processor nodesupplier
- the supplier used to obtain this node's Processor
instanceparentNames
- the name of one or more source or processor nodes whose output records this processor should receive
and processTopologyException
- if parent processor is not added yet, or if this processor's name is equal to the parent's namepublic <KIn,VIn,KOut,VOut> Topology addProcessor(String name, ProcessorSupplier<KIn,VIn,KOut,VOut> supplier, String... parentNames)
supplier
provides stores via ConnectedStoreProvider.stores()
, the provided StoreBuilder
s
will be added to the topology and connected to this processor automatically.name
- the unique name of the processor nodesupplier
- the supplier used to obtain this node's Processor
instanceparentNames
- the name of one or more source or processor nodes whose output records this processor should receive
and processTopologyException
- if parent processor is not added yet, or if this processor's name is equal to the parent's namepublic Topology addStateStore(StoreBuilder<?> storeBuilder, String... processorNames)
storeBuilder
- the storeBuilder used to obtain this state store StateStore
instanceprocessorNames
- the names of the processors that should be able to access the provided storeTopologyException
- if state store supplier is already added@Deprecated public <K,V> Topology addGlobalStore(StoreBuilder<?> storeBuilder, String sourceName, Deserializer<K> keyDeserializer, Deserializer<V> valueDeserializer, String topic, String processorName, ProcessorSupplier<K,V> stateUpdateSupplier)
addGlobalStore(StoreBuilder, String, Deserializer, Deserializer, String, String, ProcessorSupplier)
instead.StateStore
to the topology.
The StateStore
sources its data from all partitions of the provided input topic.
There will be exactly one instance of this StateStore
per Kafka Streams instance.
A SourceNode
with the provided sourceName will be added to consume the data arriving from the partitions
of the input topic.
The provided ProcessorSupplier
will be used to create an ProcessorNode
that will receive all
records forwarded from the SourceNode
.
This ProcessorNode
should be used to keep the StateStore
up-to-date.
The default TimestampExtractor
as specified in the config
is used.
storeBuilder
- user defined state store buildersourceName
- name of the SourceNode
that will be automatically addedkeyDeserializer
- the Deserializer
to deserialize keys withvalueDeserializer
- the Deserializer
to deserialize values withtopic
- the topic to source the data fromprocessorName
- the name of the ProcessorSupplier
stateUpdateSupplier
- the instance of ProcessorSupplier
TopologyException
- if the processor of state is already registered@Deprecated public <K,V> Topology addGlobalStore(StoreBuilder<?> storeBuilder, String sourceName, TimestampExtractor timestampExtractor, Deserializer<K> keyDeserializer, Deserializer<V> valueDeserializer, String topic, String processorName, ProcessorSupplier<K,V> stateUpdateSupplier)
addGlobalStore(StoreBuilder, String, TimestampExtractor, Deserializer, Deserializer, String, String, ProcessorSupplier)
instead.StateStore
to the topology.
The StateStore
sources its data from all partitions of the provided input topic.
There will be exactly one instance of this StateStore
per Kafka Streams instance.
A SourceNode
with the provided sourceName will be added to consume the data arriving from the partitions
of the input topic.
The provided ProcessorSupplier
will be used to create an ProcessorNode
that will receive all
records forwarded from the SourceNode
.
This ProcessorNode
should be used to keep the StateStore
up-to-date.
storeBuilder
- user defined key value store buildersourceName
- name of the SourceNode
that will be automatically addedtimestampExtractor
- the stateless timestamp extractor used for this source,
if not specified the default extractor defined in the configs will be usedkeyDeserializer
- the Deserializer
to deserialize keys withvalueDeserializer
- the Deserializer
to deserialize values withtopic
- the topic to source the data fromprocessorName
- the name of the ProcessorSupplier
stateUpdateSupplier
- the instance of ProcessorSupplier
TopologyException
- if the processor of state is already registeredpublic <KIn,VIn> Topology addGlobalStore(StoreBuilder<?> storeBuilder, String sourceName, Deserializer<KIn> keyDeserializer, Deserializer<VIn> valueDeserializer, String topic, String processorName, ProcessorSupplier<KIn,VIn,Void,Void> stateUpdateSupplier)
StateStore
to the topology.
The StateStore
sources its data from all partitions of the provided input topic.
There will be exactly one instance of this StateStore
per Kafka Streams instance.
A SourceNode
with the provided sourceName will be added to consume the data arriving from the partitions
of the input topic.
The provided ProcessorSupplier
will be used to create an ProcessorNode
that will receive all
records forwarded from the SourceNode
.
This ProcessorNode
should be used to keep the StateStore
up-to-date.
The default TimestampExtractor
as specified in the config
is used.
storeBuilder
- user defined state store buildersourceName
- name of the SourceNode
that will be automatically addedkeyDeserializer
- the Deserializer
to deserialize keys withvalueDeserializer
- the Deserializer
to deserialize values withtopic
- the topic to source the data fromprocessorName
- the name of the ProcessorSupplier
stateUpdateSupplier
- the instance of ProcessorSupplier
TopologyException
- if the processor of state is already registeredpublic <KIn,VIn> Topology addGlobalStore(StoreBuilder<?> storeBuilder, String sourceName, TimestampExtractor timestampExtractor, Deserializer<KIn> keyDeserializer, Deserializer<VIn> valueDeserializer, String topic, String processorName, ProcessorSupplier<KIn,VIn,Void,Void> stateUpdateSupplier)
StateStore
to the topology.
The StateStore
sources its data from all partitions of the provided input topic.
There will be exactly one instance of this StateStore
per Kafka Streams instance.
A SourceNode
with the provided sourceName will be added to consume the data arriving from the partitions
of the input topic.
The provided ProcessorSupplier
will be used to create an ProcessorNode
that will receive all
records forwarded from the SourceNode
.
This ProcessorNode
should be used to keep the StateStore
up-to-date.
storeBuilder
- user defined key value store buildersourceName
- name of the SourceNode
that will be automatically addedtimestampExtractor
- the stateless timestamp extractor used for this source,
if not specified the default extractor defined in the configs will be usedkeyDeserializer
- the Deserializer
to deserialize keys withvalueDeserializer
- the Deserializer
to deserialize values withtopic
- the topic to source the data fromprocessorName
- the name of the ProcessorSupplier
stateUpdateSupplier
- the instance of ProcessorSupplier
TopologyException
- if the processor of state is already registeredpublic Topology connectProcessorAndStateStores(String processorName, String... stateStoreNames)
processorName
- the name of the processorstateStoreNames
- the names of state stores that the processor usesTopologyException
- if the processor or a state store is unknownpublic TopologyDescription describe()
Topology
.