Class Topology


  • public class Topology
    extends Object
    A logical representation of a 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.
    • Constructor Detail

      • Topology

        public Topology()
    • Method Detail

      • addSource

        public 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. The source will use the default key deserializer and default value deserializer specified in the stream configuration. The default TimestampExtractor as specified in the config is used.
        Parameters:
        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 consume
        Returns:
        itself
        Throws:
        TopologyException - if processor is already added or if topics have already been registered by another source
      • addSource

        public 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. The source will use the default key deserializer and default value deserializer specified in the stream configuration.
        Parameters:
        timestampExtractor - the stateless timestamp extractor used for this source, if not specified the default extractor defined in the configs will be 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 consume
        Returns:
        itself
        Throws:
        TopologyException - if processor is already added or if topics have already been registered by another source
      • addSource

        public 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. The source will use the default key deserializer and default value deserializer specified in the stream configuration.
        Parameters:
        timestampExtractor - the stateless timestamp extractor used for this source, if not specified the default extractor defined in the configs will be 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 consume
        Returns:
        itself
        Throws:
        TopologyException - if processor is already added or if topics have already been registered by another source
      • addSource

        public 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. The source will use the default key deserializer and default value deserializer specified in the stream configuration.
        Parameters:
        offsetReset - the auto offset reset policy to use 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 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 consume
        Returns:
        itself
        Throws:
        TopologyException - if processor is already added or if topics have already been registered by another source
      • addSource

        public 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. The source will use the default key deserializer and default value deserializer specified in the stream configuration.
        Parameters:
        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 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 consume
        Returns:
        itself
        Throws:
        TopologyException - if processor is already added or if topics have already been registered by another source
      • addSource

        public 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. The source will use the specified key and value deserializers. The default TimestampExtractor as specified in the config is used.
        Parameters:
        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 used
        valueDeserializer - value deserializer used to read this source, if not specified the default value deserializer defined in the configs will be used
        topics - the name of one or more Kafka topics that this source is to consume
        Returns:
        itself
        Throws:
        TopologyException - if processor is already added or if topics have already been registered by another source
      • addSource

        public 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. The source will use the specified key and value deserializers. The provided de-/serializers will be used for all matched topics, so care should be taken to specify patterns for topics that share the same key-value data format. The default TimestampExtractor as specified in the config is used.
        Parameters:
        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 used
        valueDeserializer - value deserializer used to read this source, if not specified the default value deserializer defined in the configs will be used
        topicPattern - regular expression pattern to match Kafka topics that this source is to consume
        Returns:
        itself
        Throws:
        TopologyException - if processor is already added or if topics have already been registered by name
      • addSource

        public 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. The source will use the specified key and value deserializers. The provided de-/serializers will be used for all the specified topics, so care should be taken when specifying topics that share the same key-value data format.
        Parameters:
        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
        keyDeserializer - key deserializer used to read this source, if not specified the default key deserializer defined in the configs will be used
        valueDeserializer - value deserializer used to read this source, if not specified the default value deserializer defined in the configs will be used
        topics - the name of one or more Kafka topics that this source is to consume
        Returns:
        itself
        Throws:
        TopologyException - if processor is already added or if topics have already been registered by name
      • addSource

        public 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. The source will use the specified key and value deserializers. The provided de-/serializers will be used for all matched topics, so care should be taken to specify patterns for topics that share the same key-value data format.
        Parameters:
        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
        keyDeserializer - key deserializer used to read this source, if not specified the default key deserializer defined in the configs will be used
        valueDeserializer - value deserializer used to read this source, if not specified the default value deserializer defined in the configs will be used
        topicPattern - regular expression pattern to match Kafka topics that this source is to consume
        Returns:
        itself
        Throws:
        TopologyException - if processor is already added or if topics have already been registered by name
      • addSource

        public 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. The source will use the specified key and value deserializers.
        Parameters:
        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 used
        keyDeserializer - key deserializer used to read this source, if not specified the default key deserializer defined in the configs will be used
        valueDeserializer - value deserializer used to read this source, if not specified the default value deserializer defined in the configs will be used
        topics - the name of one or more Kafka topics that this source is to consume
        Returns:
        itself
        Throws:
        TopologyException - if processor is already added or if topics have already been registered by another source
      • addSource

        public 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. The source will use the specified key and value deserializers. The provided de-/serializers will be used for all matched topics, so care should be taken to specify patterns for topics that share the same key-value data format.
        Parameters:
        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 used
        keyDeserializer - key deserializer used to read this source, if not specified the default key deserializer defined in the configs will be used
        valueDeserializer - value deserializer used to read this source, if not specified the default value deserializer defined in the configs will be used
        topicPattern - regular expression pattern to match Kafka topics that this source is to consume
        Returns:
        itself
        Throws:
        TopologyException - if processor is already added or if topics have already been registered by name
      • addProcessor

        @Deprecated
        public Topology addProcessor​(String name,
                                     ProcessorSupplier supplier,
                                     String... parentNames)
        Deprecated.
        Add a new processor node that receives and processes records output by one or more parent source or processor node. Any new record output by this processor will be forwarded to its child processor or sink nodes. The supplier should always generate a new instance each time ProcessorSupplier.get() gets called. Creating a single Processor object and returning the same object reference in ProcessorSupplier.get() would be a violation of the supplier pattern and leads to runtime exceptions. If supplier provides stores via ConnectedStoreProvider.stores(), the provided StoreBuilders will be added to the topology and connected to this processor automatically.
        Parameters:
        name - the unique name of the processor node
        supplier - the supplier used to obtain this node's Processor instance
        parentNames - the name of one or more source or processor nodes whose output records this processor should receive and process
        Returns:
        itself
        Throws:
        TopologyException - if parent processor is not added yet, or if this processor's name is equal to the parent's name
      • addProcessor

        public <KIn,​VIn,​KOut,​VOut> Topology 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. Any new record output by this processor will be forwarded to its child processor or sink nodes. If supplier provides stores via ConnectedStoreProvider.stores(), the provided StoreBuilders will be added to the topology and connected to this processor automatically.
        Parameters:
        name - the unique name of the processor node
        supplier - the supplier used to obtain this node's Processor instance
        parentNames - the name of one or more source or processor nodes whose output records this processor should receive and process
        Returns:
        itself
        Throws:
        TopologyException - if parent processor is not added yet, or if this processor's name is equal to the parent's name
      • addStateStore

        public Topology addStateStore​(StoreBuilder<?> storeBuilder,
                                      String... processorNames)
        Adds a state store.
        Parameters:
        storeBuilder - the storeBuilder used to obtain this state store StateStore instance
        processorNames - the names of the processors that should be able to access the provided store
        Returns:
        itself
        Throws:
        TopologyException - if state store supplier is already added
      • addGlobalStore

        public <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. 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.

        Parameters:
        storeBuilder - user defined state store builder
        sourceName - name of the SourceNode that will be automatically added
        keyDeserializer - the Deserializer to deserialize keys with
        valueDeserializer - the Deserializer to deserialize values with
        topic - the topic to source the data from
        processorName - the name of the ProcessorSupplier
        stateUpdateSupplier - the instance of ProcessorSupplier
        Returns:
        itself
        Throws:
        TopologyException - if the processor of state is already registered
      • addGlobalStore

        public <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. 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.

        Parameters:
        storeBuilder - user defined key value store builder
        sourceName - name of the SourceNode that will be automatically added
        timestampExtractor - the stateless timestamp extractor used for this source, if not specified the default extractor defined in the configs will be used
        keyDeserializer - the Deserializer to deserialize keys with
        valueDeserializer - the Deserializer to deserialize values with
        topic - the topic to source the data from
        processorName - the name of the ProcessorSupplier
        stateUpdateSupplier - the instance of ProcessorSupplier
        Returns:
        itself
        Throws:
        TopologyException - if the processor of state is already registered
      • connectProcessorAndStateStores

        public Topology connectProcessorAndStateStores​(String processorName,
                                                       String... stateStoreNames)
        Connects the processor and the state stores.
        Parameters:
        processorName - the name of the processor
        stateStoreNames - the names of state stores that the processor uses
        Returns:
        itself
        Throws:
        TopologyException - if the processor or a state store is unknown
      • describe

        public TopologyDescription describe()
        Returns a description of the specified Topology.
        Returns:
        a description of the topology.