Interface StreamPartitioner<K,V>

Type Parameters:
K - the type of keys
V - the type of values

public interface StreamPartitioner<K,V>
Determine how records are distributed among the partitions in a Kafka topic. If not specified, the underlying producer's DefaultPartitioner will be used to determine the partition.

Kafka topics are divided into one or more partitions. Since each partition must fit on the servers that host it, so using multiple partitions allows the topic to scale beyond a size that will fit on a single machine. Partitions also enable you to use multiple instances of your topology to process in parallel all of the records on the topology's source topics.

When a topology is instantiated, each of its sources are assigned a subset of that topic's partitions. That means that only those processors in that topology instance will consume the records from those partitions. In many cases, Kafka Streams will automatically manage these instances, and adjust when new topology instances are added or removed.

Some topologies, though, need more control over which records appear in each partition. For example, some topologies that have stateful processors may want all records within a range of keys to always be delivered to and handled by the same topology instance. An upstream topology producing records to that topic can use a custom stream partitioner to precisely and consistently determine to which partition each record should be written.

To do this, create a StreamPartitioner implementation, and when you build your topology specify that custom partitioner when adding a sink for that topic.

All StreamPartitioner implementations should be stateless and a pure function so they can be shared across topic and sink nodes.

See Also:
  • Method Summary

    Modifier and Type
    Method
    Description
    partition(String topic, K key, V value, int numPartitions)
    Determine the partition number for a record with the given key and value and the current number of partitions.
  • Method Details

    • partition

      Integer partition(String topic, K key, V value, int numPartitions)
      Determine the partition number for a record with the given key and value and the current number of partitions.
      Parameters:
      topic - the topic name this record is sent to
      key - the key of the record
      value - the value of the record
      numPartitions - the total number of partitions
      Returns:
      an integer between 0 and numPartitions-1, or null if the default partitioning logic should be used