Reference Guide

This guide describes the Apache Kafka implementation of the Spring Cloud Stream Binder. It contains information about its design, usage, and configuration options, as well as information on how the Stream Cloud Stream concepts map onto Apache Kafka specific constructs. In addition, this guide explains the Kafka Streams binding capabilities of Spring Cloud Stream.

1. Apache Kafka Binder

1.1. Usage

To use Apache Kafka binder, you need to add spring-cloud-stream-binder-kafka as a dependency to your Spring Cloud Stream application, as shown in the following example for Maven:

<dependency>
  <groupId>org.springframework.cloud</groupId>
  <artifactId>spring-cloud-stream-binder-kafka</artifactId>
</dependency>

Alternatively, you can also use the Spring Cloud Stream Kafka Starter, as shown inn the following example for Maven:

<dependency>
  <groupId>org.springframework.cloud</groupId>
  <artifactId>spring-cloud-starter-stream-kafka</artifactId>
</dependency>

1.2. Overview

The following image shows a simplified diagram of how the Apache Kafka binder operates:

kafka binder
Figure 1. Kafka Binder

The Apache Kafka Binder implementation maps each destination to an Apache Kafka topic. The consumer group maps directly to the same Apache Kafka concept. Partitioning also maps directly to Apache Kafka partitions as well.

The binder currently uses the Apache Kafka kafka-clients 1.0.0 jar and is designed to be used with a broker of at least that version. This client can communicate with older brokers (see the Kafka documentation), but certain features may not be available. For example, with versions earlier than 0.11.x.x, native headers are not supported. Also, 0.11.x.x does not support the autoAddPartitions property.

1.3. Configuration Options

This section contains the configuration options used by the Apache Kafka binder.

For common configuration options and properties pertaining to binder, see the core documentation.

1.3.1. Kafka Binder Properties

spring.cloud.stream.kafka.binder.brokers

A list of brokers to which the Kafka binder connects.

Default: localhost.

spring.cloud.stream.kafka.binder.defaultBrokerPort

brokers allows hosts specified with or without port information (for example, host1,host2:port2). This sets the default port when no port is configured in the broker list.

Default: 9092.

spring.cloud.stream.kafka.binder.configuration

Key/Value map of client properties (both producers and consumer) passed to all clients created by the binder. Due to the fact that these properties are used by both producers and consumers, usage should be restricted to common properties — for example, security settings. Unknown Kafka producer or consumer properties provided through this configuration are filtered out and not allowed to propagate. Properties here supersede any properties set in boot.

Default: Empty map.

spring.cloud.stream.kafka.binder.consumerProperties

Key/Value map of arbitrary Kafka client consumer properties. In addition to support known Kafka consumer properties, unknown consumer properties are allowed here as well. Properties here supersede any properties set in boot and in the configuration property above.

Default: Empty map.

spring.cloud.stream.kafka.binder.headers

The list of custom headers that are transported by the binder. Only required when communicating with older applications (⇐ 1.3.x) with a kafka-clients version < 0.11.0.0. Newer versions support headers natively.

Default: empty.

spring.cloud.stream.kafka.binder.healthTimeout

The time to wait to get partition information, in seconds. Health reports as down if this timer expires.

Default: 10.

spring.cloud.stream.kafka.binder.requiredAcks

The number of required acks on the broker. See the Kafka documentation for the producer acks property.

Default: 1.

spring.cloud.stream.kafka.binder.minPartitionCount

Effective only if autoCreateTopics or autoAddPartitions is set. The global minimum number of partitions that the binder configures on topics on which it produces or consumes data. It can be superseded by the partitionCount setting of the producer or by the value of instanceCount * concurrency settings of the producer (if either is larger).

Default: 1.

spring.cloud.stream.kafka.binder.producerProperties

Key/Value map of arbitrary Kafka client producer properties. In addition to support known Kafka producer properties, unknown producer properties are allowed here as well. Properties here supersede any properties set in boot and in the configuration property above.

Default: Empty map.

spring.cloud.stream.kafka.binder.replicationFactor

The replication factor of auto-created topics if autoCreateTopics is active. Can be overridden on each binding.

Default: 1.

spring.cloud.stream.kafka.binder.autoCreateTopics

If set to true, the binder creates new topics automatically. If set to false, the binder relies on the topics being already configured. In the latter case, if the topics do not exist, the binder fails to start.

This setting is independent of the auto.create.topics.enable setting of the broker and does not influence it. If the server is set to auto-create topics, they may be created as part of the metadata retrieval request, with default broker settings.

Default: true.

spring.cloud.stream.kafka.binder.autoAddPartitions

If set to true, the binder creates new partitions if required. If set to false, the binder relies on the partition size of the topic being already configured. If the partition count of the target topic is smaller than the expected value, the binder fails to start.

Default: false.

spring.cloud.stream.kafka.binder.transaction.transactionIdPrefix

Enables transactions in the binder. See transaction.id in the Kafka documentation and Transactions in the spring-kafka documentation. When transactions are enabled, individual producer properties are ignored and all producers use the spring.cloud.stream.kafka.binder.transaction.producer.* properties.

Default null (no transactions)

spring.cloud.stream.kafka.binder.transaction.producer.*

Global producer properties for producers in a transactional binder. See spring.cloud.stream.kafka.binder.transaction.transactionIdPrefix and Kafka Producer Properties and the general producer properties supported by all binders.

Default: See individual producer properties.

spring.cloud.stream.kafka.binder.headerMapperBeanName

The bean name of a KafkaHeaderMapper used for mapping spring-messaging headers to and from Kafka headers. Use this, for example, if you wish to customize the trusted packages in a DefaultKafkaHeaderMapper that uses JSON deserialization for the headers.

Default: none.

1.3.2. Kafka Consumer Properties

To avoid repetition, Spring Cloud Stream supports setting values for all channels, in the format of spring.cloud.stream.default.<property>=<value>.

The following properties are available for Kafka consumers only and must be prefixed with spring.cloud.stream.kafka.bindings.<channelName>.consumer..

admin.configuration

Since version 2.1.1, this property is deprecated in favor of topic.properties, and support for it will be removed in a future version.

admin.replicas-assignment

Since version 2.1.1, this property is deprecated in favor of topic.replicas-assignment, and support for it will be removed in a future version.

admin.replication-factor

Since version 2.1.1, this property is deprecated in favor of topic.replication-factor, and support for it will be removed in a future version.

autoRebalanceEnabled

When true, topic partitions is automatically rebalanced between the members of a consumer group. When false, each consumer is assigned a fixed set of partitions based on spring.cloud.stream.instanceCount and spring.cloud.stream.instanceIndex. This requires both the spring.cloud.stream.instanceCount and spring.cloud.stream.instanceIndex properties to be set appropriately on each launched instance. The value of the spring.cloud.stream.instanceCount property must typically be greater than 1 in this case.

Default: true.

ackEachRecord

When autoCommitOffset is true, this setting dictates whether to commit the offset after each record is processed. By default, offsets are committed after all records in the batch of records returned by consumer.poll() have been processed. The number of records returned by a poll can be controlled with the max.poll.records Kafka property, which is set through the consumer configuration property. Setting this to true may cause a degradation in performance, but doing so reduces the likelihood of redelivered records when a failure occurs. Also, see the binder requiredAcks property, which also affects the performance of committing offsets.

Default: false.

autoCommitOffset

Whether to autocommit offsets when a message has been processed. If set to false, a header with the key kafka_acknowledgment of the type org.springframework.kafka.support.Acknowledgment header is present in the inbound message. Applications may use this header for acknowledging messages. See the examples section for details. When this property is set to false, Kafka binder sets the ack mode to org.springframework.kafka.listener.AbstractMessageListenerContainer.AckMode.MANUAL and the application is responsible for acknowledging records. Also see ackEachRecord.

Default: true.

autoCommitOnError

Effective only if autoCommitOffset is set to true. If set to false, it suppresses auto-commits for messages that result in errors and commits only for successful messages. It allows a stream to automatically replay from the last successfully processed message, in case of persistent failures. If set to true, it always auto-commits (if auto-commit is enabled). If not set (the default), it effectively has the same value as enableDlq, auto-committing erroneous messages if they are sent to a DLQ and not committing them otherwise.

Default: not set.

resetOffsets

Whether to reset offsets on the consumer to the value provided by startOffset. Must be false if a KafkaRebalanceListener is provided; see Using a KafkaRebalanceListener.

Default: false.

startOffset

The starting offset for new groups. Allowed values: earliest and latest. If the consumer group is set explicitly for the consumer 'binding' (through spring.cloud.stream.bindings.<channelName>.group), 'startOffset' is set to earliest. Otherwise, it is set to latest for the anonymous consumer group. Also see resetOffsets (earlier in this list).

Default: null (equivalent to earliest).

enableDlq

When set to true, it enables DLQ behavior for the consumer. By default, messages that result in errors are forwarded to a topic named error.<destination>.<group>. The DLQ topic name can be configurable by setting the dlqName property. This provides an alternative option to the more common Kafka replay scenario for the case when the number of errors is relatively small and replaying the entire original topic may be too cumbersome. See Dead-Letter Topic Processing processing for more information. Starting with version 2.0, messages sent to the DLQ topic are enhanced with the following headers: x-original-topic, x-exception-message, and x-exception-stacktrace as byte[]. Not allowed when destinationIsPattern is true.

Default: false.

configuration

Map with a key/value pair containing generic Kafka consumer properties. In addition to having Kafka consumer properties, other configuration properties can be passed here. For example some properties needed by the application such as spring.cloud.stream.kafka.bindings.input.consumer.configuration.foo=bar.

Default: Empty map.

dlqName

The name of the DLQ topic to receive the error messages.

Default: null (If not specified, messages that result in errors are forwarded to a topic named error.<destination>.<group>).

dlqProducerProperties

Using this, DLQ-specific producer properties can be set. All the properties available through kafka producer properties can be set through this property.

Default: Default Kafka producer properties.

standardHeaders

Indicates which standard headers are populated by the inbound channel adapter. Allowed values: none, id, timestamp, or both. Useful if using native deserialization and the first component to receive a message needs an id (such as an aggregator that is configured to use a JDBC message store).

Default: none

converterBeanName

The name of a bean that implements RecordMessageConverter. Used in the inbound channel adapter to replace the default MessagingMessageConverter.

Default: null

idleEventInterval

The interval, in milliseconds, between events indicating that no messages have recently been received. Use an ApplicationListener<ListenerContainerIdleEvent> to receive these events. See Example: Pausing and Resuming the Consumer for a usage example.

Default: 30000

destinationIsPattern

When true, the destination is treated as a regular expression Pattern used to match topic names by the broker. When true, topics are not provisioned, and enableDlq is not allowed, because the binder does not know the topic names during the provisioning phase. Note, the time taken to detect new topics that match the pattern is controlled by the consumer property metadata.max.age.ms, which (at the time of writing) defaults to 300,000ms (5 minutes). This can be configured using the configuration property above.

Default: false

topic.properties

A Map of Kafka topic properties used when provisioning new topics — for example, spring.cloud.stream.kafka.bindings.input.consumer.topic.properties.message.format.version=0.9.0.0

Default: none.

topic.replicas-assignment

A Map<Integer, List<Integer>> of replica assignments, with the key being the partition and the value being the assignments. Used when provisioning new topics. See the NewTopic Javadocs in the kafka-clients jar.

Default: none.

topic.replication-factor

The replication factor to use when provisioning topics. Overrides the binder-wide setting. Ignored if replicas-assignments is present.

Default: none (the binder-wide default of 1 is used).

1.3.3. Kafka Producer Properties

To avoid repetition, Spring Cloud Stream supports setting values for all channels, in the format of spring.cloud.stream.default.<property>=<value>.

The following properties are available for Kafka producers only and must be prefixed with spring.cloud.stream.kafka.bindings.<channelName>.producer..

admin.configuration

Since version 2.1.1, this property is deprecated in favor of topic.properties, and support for it will be removed in a future version.

admin.replicas-assignment

Since version 2.1.1, this property is deprecated in favor of topic.replicas-assignment, and support for it will be removed in a future version.

admin.replication-factor

Since version 2.1.1, this property is deprecated in favor of topic.replication-factor, and support for it will be removed in a future version.

bufferSize

Upper limit, in bytes, of how much data the Kafka producer attempts to batch before sending.

Default: 16384.

sync

Whether the producer is synchronous.

Default: false.

batchTimeout

How long the producer waits to allow more messages to accumulate in the same batch before sending the messages. (Normally, the producer does not wait at all and simply sends all the messages that accumulated while the previous send was in progress.) A non-zero value may increase throughput at the expense of latency.

Default: 0.

messageKeyExpression

A SpEL expression evaluated against the outgoing message used to populate the key of the produced Kafka message — for example, headers['myKey']. The payload cannot be used because, by the time this expression is evaluated, the payload is already in the form of a byte[].

Default: none.

headerPatterns

A comma-delimited list of simple patterns to match Spring messaging headers to be mapped to the Kafka Headers in the ProducerRecord. Patterns can begin or end with the wildcard character (asterisk). Patterns can be negated by prefixing with !. Matching stops after the first match (positive or negative). For example !ask,as* will pass ash but not ask. id and timestamp are never mapped.

Default: * (all headers - except the id and timestamp)

configuration

Map with a key/value pair containing generic Kafka producer properties.

Default: Empty map.

topic.properties

A Map of Kafka topic properties used when provisioning new topics — for example, spring.cloud.stream.kafka.bindings.output.producer.topic.properties.message.format.version=0.9.0.0

topic.replicas-assignment

A Map<Integer, List<Integer>> of replica assignments, with the key being the partition and the value being the assignments. Used when provisioning new topics. See the NewTopic Javadocs in the kafka-clients jar.

Default: none.

topic.replication-factor

The replication factor to use when provisioning topics. Overrides the binder-wide setting. Ignored if replicas-assignments is present.

Default: none (the binder-wide default of 1 is used).

The Kafka binder uses the partitionCount setting of the producer as a hint to create a topic with the given partition count (in conjunction with the minPartitionCount, the maximum of the two being the value being used). Exercise caution when configuring both minPartitionCount for a binder and partitionCount for an application, as the larger value is used. If a topic already exists with a smaller partition count and autoAddPartitions is disabled (the default), the binder fails to start. If a topic already exists with a smaller partition count and autoAddPartitions is enabled, new partitions are added. If a topic already exists with a larger number of partitions than the maximum of (minPartitionCount or partitionCount), the existing partition count is used.
compression

Set the compression.type producer property. Supported values are none, gzip, snappy and lz4. If you override the kafka-clients jar to 2.1.0 (or later), as discussed in the Spring for Apache Kafka documentation, and wish to use zstd compression, use spring.cloud.stream.kafka.bindings.<binding-name>.producer.configuration.compression.type=zstd.

Default: none.

1.3.4. Usage examples

In this section, we show the use of the preceding properties for specific scenarios.

Example: Setting autoCommitOffset to false and Relying on Manual Acking

This example illustrates how one may manually acknowledge offsets in a consumer application.

This example requires that spring.cloud.stream.kafka.bindings.input.consumer.autoCommitOffset be set to false. Use the corresponding input channel name for your example.

@SpringBootApplication
@EnableBinding(Sink.class)
public class ManuallyAcknowdledgingConsumer {

 public static void main(String[] args) {
     SpringApplication.run(ManuallyAcknowdledgingConsumer.class, args);
 }

 @StreamListener(Sink.INPUT)
 public void process(Message<?> message) {
     Acknowledgment acknowledgment = message.getHeaders().get(KafkaHeaders.ACKNOWLEDGMENT, Acknowledgment.class);
     if (acknowledgment != null) {
         System.out.println("Acknowledgment provided");
         acknowledgment.acknowledge();
     }
 }
}
Example: Security Configuration

Apache Kafka 0.9 supports secure connections between client and brokers. To take advantage of this feature, follow the guidelines in the Apache Kafka Documentation as well as the Kafka 0.9 security guidelines from the Confluent documentation. Use the spring.cloud.stream.kafka.binder.configuration option to set security properties for all clients created by the binder.

For example, to set security.protocol to SASL_SSL, set the following property:

spring.cloud.stream.kafka.binder.configuration.security.protocol=SASL_SSL

All the other security properties can be set in a similar manner.

When using Kerberos, follow the instructions in the reference documentation for creating and referencing the JAAS configuration.

Spring Cloud Stream supports passing JAAS configuration information to the application by using a JAAS configuration file and using Spring Boot properties.

Using JAAS Configuration Files

The JAAS and (optionally) krb5 file locations can be set for Spring Cloud Stream applications by using system properties. The following example shows how to launch a Spring Cloud Stream application with SASL and Kerberos by using a JAAS configuration file:

 java -Djava.security.auth.login.config=/path.to/kafka_client_jaas.conf -jar log.jar \
   --spring.cloud.stream.kafka.binder.brokers=secure.server:9092 \
   --spring.cloud.stream.bindings.input.destination=stream.ticktock \
   --spring.cloud.stream.kafka.binder.configuration.security.protocol=SASL_PLAINTEXT
Using Spring Boot Properties

As an alternative to having a JAAS configuration file, Spring Cloud Stream provides a mechanism for setting up the JAAS configuration for Spring Cloud Stream applications by using Spring Boot properties.

The following properties can be used to configure the login context of the Kafka client:

spring.cloud.stream.kafka.binder.jaas.loginModule

The login module name. Not necessary to be set in normal cases.

Default: com.sun.security.auth.module.Krb5LoginModule.

spring.cloud.stream.kafka.binder.jaas.controlFlag

The control flag of the login module.

Default: required.

spring.cloud.stream.kafka.binder.jaas.options

Map with a key/value pair containing the login module options.

Default: Empty map.

The following example shows how to launch a Spring Cloud Stream application with SASL and Kerberos by using Spring Boot configuration properties:

 java --spring.cloud.stream.kafka.binder.brokers=secure.server:9092 \
   --spring.cloud.stream.bindings.input.destination=stream.ticktock \
   --spring.cloud.stream.kafka.binder.autoCreateTopics=false \
   --spring.cloud.stream.kafka.binder.configuration.security.protocol=SASL_PLAINTEXT \
   --spring.cloud.stream.kafka.binder.jaas.options.useKeyTab=true \
   --spring.cloud.stream.kafka.binder.jaas.options.storeKey=true \
   --spring.cloud.stream.kafka.binder.jaas.options.keyTab=/etc/security/keytabs/kafka_client.keytab \
   --spring.cloud.stream.kafka.binder.jaas.options.principal=kafka-client-1@EXAMPLE.COM

The preceding example represents the equivalent of the following JAAS file:

KafkaClient {
    com.sun.security.auth.module.Krb5LoginModule required
    useKeyTab=true
    storeKey=true
    keyTab="/etc/security/keytabs/kafka_client.keytab"
    principal="[email protected]";
};

If the topics required already exist on the broker or will be created by an administrator, autocreation can be turned off and only client JAAS properties need to be sent.

Do not mix JAAS configuration files and Spring Boot properties in the same application. If the -Djava.security.auth.login.config system property is already present, Spring Cloud Stream ignores the Spring Boot properties.
Be careful when using the autoCreateTopics and autoAddPartitions with Kerberos. Usually, applications may use principals that do not have administrative rights in Kafka and Zookeeper. Consequently, relying on Spring Cloud Stream to create/modify topics may fail. In secure environments, we strongly recommend creating topics and managing ACLs administratively by using Kafka tooling.
Example: Pausing and Resuming the Consumer

If you wish to suspend consumption but not cause a partition rebalance, you can pause and resume the consumer. This is facilitated by adding the Consumer as a parameter to your @StreamListener. To resume, you need an ApplicationListener for ListenerContainerIdleEvent instances. The frequency at which events are published is controlled by the idleEventInterval property. Since the consumer is not thread-safe, you must call these methods on the calling thread.

The following simple application shows how to pause and resume:

@SpringBootApplication
@EnableBinding(Sink.class)
public class Application {

	public static void main(String[] args) {
		SpringApplication.run(Application.class, args);
	}

	@StreamListener(Sink.INPUT)
	public void in(String in, @Header(KafkaHeaders.CONSUMER) Consumer<?, ?> consumer) {
		System.out.println(in);
		consumer.pause(Collections.singleton(new TopicPartition("myTopic", 0)));
	}

	@Bean
	public ApplicationListener<ListenerContainerIdleEvent> idleListener() {
		return event -> {
			System.out.println(event);
			if (event.getConsumer().paused().size() > 0) {
				event.getConsumer().resume(event.getConsumer().paused());
			}
		};
	}

}

1.4. Error Channels

Starting with version 1.3, the binder unconditionally sends exceptions to an error channel for each consumer destination and can also be configured to send async producer send failures to an error channel. See [spring-cloud-stream-overview-error-handling] for more information.

The payload of the ErrorMessage for a send failure is a KafkaSendFailureException with properties:

  • failedMessage: The Spring Messaging Message<?> that failed to be sent.

  • record: The raw ProducerRecord that was created from the failedMessage

There is no automatic handling of producer exceptions (such as sending to a Dead-Letter queue). You can consume these exceptions with your own Spring Integration flow.

1.5. Kafka Metrics

Kafka binder module exposes the following metrics:

spring.cloud.stream.binder.kafka.offset: This metric indicates how many messages have not been yet consumed from a given binder’s topic by a given consumer group. The metrics provided are based on the Mircometer metrics library. The metric contains the consumer group information, topic and the actual lag in committed offset from the latest offset on the topic. This metric is particularly useful for providing auto-scaling feedback to a PaaS platform.

1.6. Tombstone Records (null record values)

When using compacted topics, a record with a null value (also called a tombstone record) represents the deletion of a key. To receive such messages in a @StreamListener method, the parameter must be marked as not required to receive a null value argument.

@StreamListener(Sink.INPUT)
public void in(@Header(KafkaHeaders.RECEIVED_MESSAGE_KEY) byte[] key,
               @Payload(required = false) Customer customer) {
    // customer is null if a tombstone record
    ...
}

1.7. Using a KafkaRebalanceListener

Applications may wish to seek topics/partitions to arbitrary offsets when the partitions are initially assigned, or perform other operations on the consumer. Starting with version 2.1, if you provide a single KafkaRebalanceListener bean in the application context, it will be wired into all Kafka consumer bindings.

public interface KafkaBindingRebalanceListener {

	/**
	 * Invoked by the container before any pending offsets are committed.
	 * @param bindingName the name of the binding.
	 * @param consumer the consumer.
	 * @param partitions the partitions.
	 */
	default void onPartitionsRevokedBeforeCommit(String bindingName, Consumer<?, ?> consumer,
			Collection<TopicPartition> partitions) {

	}

	/**
	 * Invoked by the container after any pending offsets are committed.
	 * @param bindingName the name of the binding.
	 * @param consumer the consumer.
	 * @param partitions the partitions.
	 */
	default void onPartitionsRevokedAfterCommit(String bindingName, Consumer<?, ?> consumer, Collection<TopicPartition> partitions) {

	}

	/**
	 * Invoked when partitions are initially assigned or after a rebalance.
	 * Applications might only want to perform seek operations on an initial assignment.
	 * @param bindingName the name of the binding.
	 * @param consumer the consumer.
	 * @param partitions the partitions.
	 * @param initial true if this is the initial assignment.
	 */
	default void onPartitionsAssigned(String bindingName, Consumer<?, ?> consumer, Collection<TopicPartition> partitions,
			boolean initial) {

	}

}

You cannot set the resetOffsets consumer property to true when you provide a rebalance listener.

1.8. Dead-Letter Topic Processing

Because you cannot anticipate how users would want to dispose of dead-lettered messages, the framework does not provide any standard mechanism to handle them. If the reason for the dead-lettering is transient, you may wish to route the messages back to the original topic. However, if the problem is a permanent issue, that could cause an infinite loop. The sample Spring Boot application within this topic is an example of how to route those messages back to the original topic, but it moves them to a “parking lot” topic after three attempts. The application is another spring-cloud-stream application that reads from the dead-letter topic. It terminates when no messages are received for 5 seconds.

The examples assume the original destination is so8400out and the consumer group is so8400.

There are a couple of strategies to consider:

  • Consider running the rerouting only when the main application is not running. Otherwise, the retries for transient errors are used up very quickly.

  • Alternatively, use a two-stage approach: Use this application to route to a third topic and another to route from there back to the main topic.

The following code listings show the sample application:

application.properties
spring.cloud.stream.bindings.input.group=so8400replay
spring.cloud.stream.bindings.input.destination=error.so8400out.so8400

spring.cloud.stream.bindings.output.destination=so8400out

spring.cloud.stream.bindings.parkingLot.destination=so8400in.parkingLot

spring.cloud.stream.kafka.binder.configuration.auto.offset.reset=earliest

spring.cloud.stream.kafka.binder.headers=x-retries
Application
@SpringBootApplication
@EnableBinding(TwoOutputProcessor.class)
public class ReRouteDlqKApplication implements CommandLineRunner {

    private static final String X_RETRIES_HEADER = "x-retries";

    public static void main(String[] args) {
        SpringApplication.run(ReRouteDlqKApplication.class, args).close();
    }

    private final AtomicInteger processed = new AtomicInteger();

    @Autowired
    private MessageChannel parkingLot;

    @StreamListener(Processor.INPUT)
    @SendTo(Processor.OUTPUT)
    public Message<?> reRoute(Message<?> failed) {
        processed.incrementAndGet();
        Integer retries = failed.getHeaders().get(X_RETRIES_HEADER, Integer.class);
        if (retries == null) {
            System.out.println("First retry for " + failed);
            return MessageBuilder.fromMessage(failed)
                    .setHeader(X_RETRIES_HEADER, new Integer(1))
                    .setHeader(BinderHeaders.PARTITION_OVERRIDE,
                            failed.getHeaders().get(KafkaHeaders.RECEIVED_PARTITION_ID))
                    .build();
        }
        else if (retries.intValue() < 3) {
            System.out.println("Another retry for " + failed);
            return MessageBuilder.fromMessage(failed)
                    .setHeader(X_RETRIES_HEADER, new Integer(retries.intValue() + 1))
                    .setHeader(BinderHeaders.PARTITION_OVERRIDE,
                            failed.getHeaders().get(KafkaHeaders.RECEIVED_PARTITION_ID))
                    .build();
        }
        else {
            System.out.println("Retries exhausted for " + failed);
            parkingLot.send(MessageBuilder.fromMessage(failed)
                    .setHeader(BinderHeaders.PARTITION_OVERRIDE,
                            failed.getHeaders().get(KafkaHeaders.RECEIVED_PARTITION_ID))
                    .build());
        }
        return null;
    }

    @Override
    public void run(String... args) throws Exception {
        while (true) {
            int count = this.processed.get();
            Thread.sleep(5000);
            if (count == this.processed.get()) {
                System.out.println("Idle, terminating");
                return;
            }
        }
    }

    public interface TwoOutputProcessor extends Processor {

        @Output("parkingLot")
        MessageChannel parkingLot();

    }

}

1.9. Partitioning with the Kafka Binder

Apache Kafka supports topic partitioning natively.

Sometimes it is advantageous to send data to specific partitions — for example, when you want to strictly order message processing (all messages for a particular customer should go to the same partition).

The following example shows how to configure the producer and consumer side:

@SpringBootApplication
@EnableBinding(Source.class)
public class KafkaPartitionProducerApplication {

    private static final Random RANDOM = new Random(System.currentTimeMillis());

    private static final String[] data = new String[] {
            "foo1", "bar1", "qux1",
            "foo2", "bar2", "qux2",
            "foo3", "bar3", "qux3",
            "foo4", "bar4", "qux4",
            };

    public static void main(String[] args) {
        new SpringApplicationBuilder(KafkaPartitionProducerApplication.class)
            .web(false)
            .run(args);
    }

    @InboundChannelAdapter(channel = Source.OUTPUT, poller = @Poller(fixedRate = "5000"))
    public Message<?> generate() {
        String value = data[RANDOM.nextInt(data.length)];
        System.out.println("Sending: " + value);
        return MessageBuilder.withPayload(value)
                .setHeader("partitionKey", value)
                .build();
    }

}
application.yml
spring:
  cloud:
    stream:
      bindings:
        output:
          destination: partitioned.topic
          producer:
            partition-key-expression: headers['partitionKey']
            partition-count: 12
The topic must be provisioned to have enough partitions to achieve the desired concurrency for all consumer groups. The above configuration supports up to 12 consumer instances (6 if their concurrency is 2, 4 if their concurrency is 3, and so on). It is generally best to “over-provision” the partitions to allow for future increases in consumers or concurrency.
The preceding configuration uses the default partitioning (key.hashCode() % partitionCount). This may or may not provide a suitably balanced algorithm, depending on the key values. You can override this default by using the partitionSelectorExpression or partitionSelectorClass properties.

Since partitions are natively handled by Kafka, no special configuration is needed on the consumer side. Kafka allocates partitions across the instances.

The following Spring Boot application listens to a Kafka stream and prints (to the console) the partition ID to which each message goes:

@SpringBootApplication
@EnableBinding(Sink.class)
public class KafkaPartitionConsumerApplication {

    public static void main(String[] args) {
        new SpringApplicationBuilder(KafkaPartitionConsumerApplication.class)
            .web(false)
            .run(args);
    }

    @StreamListener(Sink.INPUT)
    public void listen(@Payload String in, @Header(KafkaHeaders.RECEIVED_PARTITION_ID) int partition) {
        System.out.println(in + " received from partition " + partition);
    }

}
application.yml
spring:
  cloud:
    stream:
      bindings:
        input:
          destination: partitioned.topic
          group: myGroup

You can add instances as needed. Kafka rebalances the partition allocations. If the instance count (or instance count * concurrency) exceeds the number of partitions, some consumers are idle.

2. Kafka Streams Binder

2.1. Usage

For using the Kafka Streams binder, you just need to add it to your Spring Cloud Stream application, using the following Maven coordinates:

<dependency>
  <groupId>org.springframework.cloud</groupId>
  <artifactId>spring-cloud-stream-binder-kafka-streams</artifactId>
</dependency>

2.2. Overview

Spring Cloud Stream’s Apache Kafka support also includes a binder implementation designed explicitly for Apache Kafka Streams binding. With this native integration, a Spring Cloud Stream "processor" application can directly use the Apache Kafka Streams APIs in the core business logic.

Kafka Streams binder implementation builds on the foundation provided by the Kafka Streams in Spring Kafka project.

Kafka Streams binder provides binding capabilities for the three major types in Kafka Streams - KStream, KTable and GlobalKTable.

As part of this native integration, the high-level Streams DSL provided by the Kafka Streams API is available for use in the business logic.

An early version of the Processor API support is available as well.

As noted early-on, Kafka Streams support in Spring Cloud Stream is strictly only available for use in the Processor model. A model in which the messages read from an inbound topic, business processing can be applied, and the transformed messages can be written to an outbound topic. It can also be used in Processor applications with a no-outbound destination.

2.2.1. Streams DSL

This application consumes data from a Kafka topic (e.g., words), computes word count for each unique word in a 5 seconds time window, and the computed results are sent to a downstream topic (e.g., counts) for further processing.

@SpringBootApplication
@EnableBinding(KafkaStreamsProcessor.class)
public class WordCountProcessorApplication {

	@StreamListener("input")
	@SendTo("output")
	public KStream<?, WordCount> process(KStream<?, String> input) {
		return input
                .flatMapValues(value -> Arrays.asList(value.toLowerCase().split("\\W+")))
                .groupBy((key, value) -> value)
                .windowedBy(TimeWindows.of(5000))
                .count(Materialized.as("WordCounts-multi"))
                .toStream()
                .map((key, value) -> new KeyValue<>(null, new WordCount(key.key(), value, new Date(key.window().start()), new Date(key.window().end()))));
    }

	public static void main(String[] args) {
		SpringApplication.run(WordCountProcessorApplication.class, args);
	}

Once built as a uber-jar (e.g., wordcount-processor.jar), you can run the above example like the following.

java -jar wordcount-processor.jar  --spring.cloud.stream.bindings.input.destination=words --spring.cloud.stream.bindings.output.destination=counts

This application will consume messages from the Kafka topic words and the computed results are published to an output topic counts.

Spring Cloud Stream will ensure that the messages from both the incoming and outgoing topics are automatically bound as KStream objects. As a developer, you can exclusively focus on the business aspects of the code, i.e. writing the logic required in the processor. Setting up the Streams DSL specific configuration required by the Kafka Streams infrastructure is automatically handled by the framework.

2.3. Configuration Options

This section contains the configuration options used by the Kafka Streams binder.

For common configuration options and properties pertaining to binder, refer to the core documentation.

2.3.1. Kafka Streams Properties

The following properties are available at the binder level and must be prefixed with spring.cloud.stream.kafka.streams.binder.

configuration

Map with a key/value pair containing properties pertaining to Apache Kafka Streams API. This property must be prefixed with spring.cloud.stream.kafka.streams.binder.. Following are some examples of using this property.

spring.cloud.stream.kafka.streams.binder.configuration.default.key.serde=org.apache.kafka.common.serialization.Serdes$StringSerde
spring.cloud.stream.kafka.streams.binder.configuration.default.value.serde=org.apache.kafka.common.serialization.Serdes$StringSerde
spring.cloud.stream.kafka.streams.binder.configuration.commit.interval.ms=1000

For more information about all the properties that may go into streams configuration, see StreamsConfig JavaDocs in Apache Kafka Streams docs.

brokers

Broker URL

Default: localhost

zkNodes

Zookeeper URL

Default: localhost

serdeError

Deserialization error handler type. Possible values are - logAndContinue, logAndFail or sendToDlq

Default: logAndFail

applicationId

Convenient way to set the application.id for the Kafka Streams application globally at the binder level. If the application contains multiple StreamListener methods, then application.id should be set at the binding level per input binding.

Default: none

The following properties are only available for Kafka Streams producers and must be prefixed with spring.cloud.stream.kafka.streams.bindings.<binding name>.producer. For convenience, if there multiple output bindings and they all require a common value, that can be configured by using the prefix spring.cloud.stream.kafka.streams.default.producer..

keySerde

key serde to use

Default: none.

valueSerde

value serde to use

Default: none.

useNativeEncoding

flag to enable native encoding

Default: false.

The following properties are available for Kafka Streams consumers and must be prefixed with spring.cloud.stream.kafka.streams.bindings.<binding-name>.consumer. For convenience, if there are multiple input bindings and they all require a common value, that can be configured by using the prefix spring.cloud.stream.kafka.streams.default.consumer..

applicationId

Setting application.id per input binding.

Default: none

keySerde

key serde to use

Default: none.

valueSerde

value serde to use

Default: none.

materializedAs

state store to materialize when using incoming KTable types

Default: none.

useNativeDecoding

flag to enable native decoding

Default: false.

dlqName

DLQ topic name.

Default: none.

startOffset

Offset to start from if there is no committed offset to consume from. This is mostly used when the consumer is consuming from a topic for the first time. Kafka Streams uses earliest as the default strategy and the binder uses the same default. This can be overridden to latest using this property.

Default: earliest.

Note: Using resetOffsets on the consumer does not have any effect on Kafka Streams binder. Unlike the message channel based binder, Kafka Streams binder does not seek to beginning or end on demand.

2.3.2. TimeWindow properties:

Windowing is an important concept in stream processing applications. Following properties are available to configure time-window computations.

spring.cloud.stream.kafka.streams.timeWindow.length

When this property is given, you can autowire a TimeWindows bean into the application. The value is expressed in milliseconds.

Default: none.

spring.cloud.stream.kafka.streams.timeWindow.advanceBy

Value is given in milliseconds.

Default: none.

2.4. Multiple Input Bindings

For use cases that requires multiple incoming KStream objects or a combination of KStream and KTable objects, the Kafka Streams binder provides multiple bindings support.

Let’s see it in action.

2.4.1. Multiple Input Bindings as a Sink

@EnableBinding(KStreamKTableBinding.class)
.....
.....
@StreamListener
public void process(@Input("inputStream") KStream<String, PlayEvent> playEvents,
                    @Input("inputTable") KTable<Long, Song> songTable) {
                    ....
                    ....
}

interface KStreamKTableBinding {

    @Input("inputStream")
    KStream<?, ?> inputStream();

    @Input("inputTable")
    KTable<?, ?> inputTable();
}

In the above example, the application is written as a sink, i.e. there are no output bindings and the application has to decide concerning downstream processing. When you write applications in this style, you might want to send the information downstream or store them in a state store (See below for Queryable State Stores).

In the case of incoming KTable, if you want to materialize the computations to a state store, you have to express it through the following property.

spring.cloud.stream.kafka.streams.bindings.inputTable.consumer.materializedAs: all-songs

The above example shows the use of KTable as an input binding. The binder also supports input bindings for GlobalKTable. GlobalKTable binding is useful when you have to ensure that all instances of your application has access to the data updates from the topic. KTable and GlobalKTable bindings are only available on the input. Binder supports both input and output bindings for KStream.

2.5. Multiple Input Bindings as a Processor

@EnableBinding(KStreamKTableBinding.class)
....
....

@StreamListener
@SendTo("output")
public KStream<String, Long> process(@Input("input") KStream<String, Long> userClicksStream,
                                     @Input("inputTable") KTable<String, String> userRegionsTable) {
....
....
}

interface KStreamKTableBinding extends KafkaStreamsProcessor {

    @Input("inputX")
    KTable<?, ?> inputTable();
}

2.6. Multiple Output Bindings (aka Branching)

Kafka Streams allow outbound data to be split into multiple topics based on some predicates. The Kafka Streams binder provides support for this feature without compromising the programming model exposed through StreamListener in the end user application.

You can write the application in the usual way as demonstrated above in the word count example. However, when using the branching feature, you are required to do a few things. First, you need to make sure that your return type is KStream[] instead of a regular KStream. Second, you need to use the SendTo annotation containing the output bindings in the order (see example below). For each of these output bindings, you need to configure destination, content-type etc., complying with the standard Spring Cloud Stream expectations.

Here is an example:

@EnableBinding(KStreamProcessorWithBranches.class)
@EnableAutoConfiguration
public static class WordCountProcessorApplication {

    @Autowired
    private TimeWindows timeWindows;

    @StreamListener("input")
    @SendTo({"output1","output2","output3})
    public KStream<?, WordCount>[] process(KStream<Object, String> input) {

			Predicate<Object, WordCount> isEnglish = (k, v) -> v.word.equals("english");
			Predicate<Object, WordCount> isFrench =  (k, v) -> v.word.equals("french");
			Predicate<Object, WordCount> isSpanish = (k, v) -> v.word.equals("spanish");

			return input
					.flatMapValues(value -> Arrays.asList(value.toLowerCase().split("\\W+")))
					.groupBy((key, value) -> value)
					.windowedBy(timeWindows)
					.count(Materialized.as("WordCounts-1"))
					.toStream()
					.map((key, value) -> new KeyValue<>(null, new WordCount(key.key(), value, new Date(key.window().start()), new Date(key.window().end()))))
					.branch(isEnglish, isFrench, isSpanish);
    }

    interface KStreamProcessorWithBranches {

    		@Input("input")
    		KStream<?, ?> input();

    		@Output("output1")
    		KStream<?, ?> output1();

    		@Output("output2")
    		KStream<?, ?> output2();

    		@Output("output3")
    		KStream<?, ?> output3();
    	}
}

Properties:

spring.cloud.stream.bindings.output1.contentType: application/json
spring.cloud.stream.bindings.output2.contentType: application/json
spring.cloud.stream.bindings.output3.contentType: application/json
spring.cloud.stream.kafka.streams.binder.configuration.commit.interval.ms: 1000
spring.cloud.stream.kafka.streams.binder.configuration:
  default.key.serde: org.apache.kafka.common.serialization.Serdes$StringSerde
  default.value.serde: org.apache.kafka.common.serialization.Serdes$StringSerde
spring.cloud.stream.bindings.output1:
  destination: foo
  producer:
    headerMode: raw
spring.cloud.stream.bindings.output2:
  destination: bar
  producer:
    headerMode: raw
spring.cloud.stream.bindings.output3:
  destination: fox
  producer:
    headerMode: raw
spring.cloud.stream.bindings.input:
  destination: words
  consumer:
    headerMode: raw

2.7. Record Value Conversion

Kafka Streams binder can marshal producer/consumer values based on a content type and the converters provided out of the box in Spring Cloud Stream.

It is typical for Kafka Streams applications to provide Serde classes. Therefore, it may be more natural to rely on the SerDe facilities provided by the Apache Kafka Streams library itself for data conversion on inbound and outbound rather than rely on the content-type conversions offered by the binder. On the other hand, you might be already familiar with the content-type conversion patterns provided by Spring Cloud Stream and would like to continue using that for inbound and outbound conversions.

Both the options are supported in the Kafka Streams binder implementation. See below for more details.

Outbound serialization

If native encoding is disabled (which is the default), then the framework will convert the message using the contentType set by the user (otherwise, the default application/json will be applied). It will ignore any SerDe set on the outbound in this case for outbound serialization.

Here is the property to set the contentType on the outbound.

spring.cloud.stream.bindings.output.contentType: application/json

Here is the property to enable native encoding.

spring.cloud.stream.bindings.output.nativeEncoding: true

If native encoding is enabled on the output binding (user has to enable it as above explicitly), then the framework will skip any form of automatic message conversion on the outbound. In that case, it will switch to the Serde set by the user. The valueSerde property set on the actual output binding will be used. Here is an example.

spring.cloud.stream.kafka.streams.bindings.output.producer.valueSerde: org.apache.kafka.common.serialization.Serdes$StringSerde

If this property is not set, then it will use the "default" SerDe: spring.cloud.stream.kafka.streams.binder.configuration.default.value.serde.

It is worth to mention that Kafka Streams binder does not serialize the keys on outbound - it simply relies on Kafka itself. Therefore, you either have to specify the keySerde property on the binding or it will default to the application-wide common keySerde.

Binding level key serde:

spring.cloud.stream.kafka.streams.bindings.output.producer.keySerde

Common Key serde:

spring.cloud.stream.kafka.streams.binder.configuration.default.key.serde

If branching is used, then you need to use multiple output bindings. For example,

interface KStreamProcessorWithBranches {

    		@Input("input")
    		KStream<?, ?> input();

    		@Output("output1")
    		KStream<?, ?> output1();

    		@Output("output2")
    		KStream<?, ?> output2();

    		@Output("output3")
    		KStream<?, ?> output3();
    	}

If nativeEncoding is set, then you can set different SerDe’s on individual output bindings as below.

spring.cloud.stream.kafka.streams.bindings.output1.producer.valueSerde=IntegerSerde
spring.cloud.stream.kafka.streams.bindings.output2.producer.valueSerde=StringSerde
spring.cloud.stream.kafka.streams.bindings.output3.producer.valueSerde=JsonSerde

Then if you have SendTo like this, @SendTo({"output1", "output2", "output3"}), the KStream[] from the branches are applied with proper SerDe objects as defined above. If you are not enabling nativeEncoding, you can then set different contentType values on the output bindings as below. In that case, the framework will use the appropriate message converter to convert the messages before sending to Kafka.

spring.cloud.stream.bindings.output1.contentType: application/json
spring.cloud.stream.bindings.output2.contentType: application/java-serialzied-object
spring.cloud.stream.bindings.output3.contentType: application/octet-stream
Inbound Deserialization

Similar rules apply to data deserialization on the inbound.

If native decoding is disabled (which is the default), then the framework will convert the message using the contentType set by the user (otherwise, the default application/json will be applied). It will ignore any SerDe set on the inbound in this case for inbound deserialization.

Here is the property to set the contentType on the inbound.

spring.cloud.stream.bindings.input.contentType: application/json

Here is the property to enable native decoding.

spring.cloud.stream.bindings.input.nativeDecoding: true

If native decoding is enabled on the input binding (user has to enable it as above explicitly), then the framework will skip doing any message conversion on the inbound. In that case, it will switch to the SerDe set by the user. The valueSerde property set on the actual output binding will be used. Here is an example.

spring.cloud.stream.kafka.streams.bindings.input.consumer.valueSerde: org.apache.kafka.common.serialization.Serdes$StringSerde

If this property is not set, it will use the default SerDe: spring.cloud.stream.kafka.streams.binder.configuration.default.value.serde.

It is worth to mention that Kafka Streams binder does not deserialize the keys on inbound - it simply relies on Kafka itself. Therefore, you either have to specify the keySerde property on the binding or it will default to the application-wide common keySerde.

Binding level key serde:

spring.cloud.stream.kafka.streams.bindings.input.consumer.keySerde

Common Key serde:

spring.cloud.stream.kafka.streams.binder.configuration.default.key.serde

As in the case of KStream branching on the outbound, the benefit of setting value SerDe per binding is that if you have multiple input bindings (multiple KStreams object) and they all require separate value SerDe’s, then you can configure them individually. If you use the common configuration approach, then this feature won’t be applicable.

2.8. Error Handling

Apache Kafka Streams provide the capability for natively handling exceptions from deserialization errors. For details on this support, please see this Out of the box, Apache Kafka Streams provide two kinds of deserialization exception handlers - logAndContinue and logAndFail. As the name indicates, the former will log the error and continue processing the next records and the latter will log the error and fail. LogAndFail is the default deserialization exception handler.

2.9. Handling Deserialization Exceptions

Kafka Streams binder supports a selection of exception handlers through the following properties.

spring.cloud.stream.kafka.streams.binder.serdeError: logAndContinue

In addition to the above two deserialization exception handlers, the binder also provides a third one for sending the erroneous records (poison pills) to a DLQ topic. Here is how you enable this DLQ exception handler.

spring.cloud.stream.kafka.streams.binder.serdeError: sendToDlq

When the above property is set, all the deserialization error records are automatically sent to the DLQ topic.

spring.cloud.stream.kafka.streams.bindings.input.consumer.dlqName: foo-dlq

If this is set, then the error records are sent to the topic foo-dlq. If this is not set, then it will create a DLQ topic with the name error.<input-topic-name>.<group-name>.

A couple of things to keep in mind when using the exception handling feature in Kafka Streams binder.

  • The property spring.cloud.stream.kafka.streams.binder.serdeError is applicable for the entire application. This implies that if there are multiple StreamListener methods in the same application, this property is applied to all of them.

  • The exception handling for deserialization works consistently with native deserialization and framework provided message conversion.

2.9.1. Handling Non-Deserialization Exceptions

For general error handling in Kafka Streams binder, it is up to the end user applications to handle application level errors. As a side effect of providing a DLQ for deserialization exception handlers, Kafka Streams binder provides a way to get access to the DLQ sending bean directly from your application. Once you get access to that bean, you can programmatically send any exception records from your application to the DLQ.

It continues to remain hard to robust error handling using the high-level DSL; Kafka Streams doesn’t natively support error handling yet.

However, when you use the low-level Processor API in your application, there are options to control this behavior. See below.

@Autowired
private SendToDlqAndContinue dlqHandler;

@StreamListener("input")
@SendTo("output")
public KStream<?, WordCount> process(KStream<Object, String> input) {

    input.process(() -> new Processor() {
    			ProcessorContext context;

    			@Override
    			public void init(ProcessorContext context) {
    				this.context = context;
    			}

    			@Override
    			public void process(Object o, Object o2) {

    			    try {
    			        .....
    			        .....
    			    }
    			    catch(Exception e) {
    			        //explicitly provide the kafka topic corresponding to the input binding as the first argument.
                        //DLQ handler will correctly map to the dlq topic from the actual incoming destination.
                        dlqHandler.sendToDlq("topic-name", (byte[]) o1, (byte[]) o2, context.partition());
    			    }
    			}

    			.....
    			.....
    });
}

2.10. State Store

State store is created automatically by Kafka Streams when the DSL is used. When processor API is used, you need to register a state store manually. In order to do so, you can use KafkaStreamsStateStore annotation. You can specify the name and type of the store, flags to control log and disabling cache, etc. Once the store is created by the binder during the bootstrapping phase, you can access this state store through the processor API. Below are some primitives for doing this.

Creating a state store:

@KafkaStreamsStateStore(name="mystate", type= KafkaStreamsStateStoreProperties.StoreType.WINDOW, lengthMs=300000)
public void process(KStream<Object, Product> input) {
    ...
}

Accessing the state store:

Processor<Object, Product>() {

    WindowStore<Object, String> state;

    @Override
    public void init(ProcessorContext processorContext) {
        state = (WindowStore)processorContext.getStateStore("mystate");
    }
    ...
}

2.11. Interactive Queries

As part of the public Kafka Streams binder API, we expose a class called InteractiveQueryService. You can access this as a Spring bean in your application. An easy way to get access to this bean from your application is to "autowire" the bean.

@Autowired
private InteractiveQueryService interactiveQueryService;

Once you gain access to this bean, then you can query for the particular state-store that you are interested. See below.

ReadOnlyKeyValueStore<Object, Object> keyValueStore =
						interactiveQueryService.getQueryableStoreType("my-store", QueryableStoreTypes.keyValueStore());

If there are multiple instances of the kafka streams application running, then before you can query them interactively, you need to identify which application instance hosts the key. InteractiveQueryService API provides methods for identifying the host information.

In order for this to work, you must configure the property application.server as below:

spring.cloud.stream.kafka.streams.binder.configuration.application.server: <server>:<port>

Here are some code snippets:

org.apache.kafka.streams.state.HostInfo hostInfo = interactiveQueryService.getHostInfo("store-name",
						key, keySerializer);

if (interactiveQueryService.getCurrentHostInfo().equals(hostInfo)) {

    //query from the store that is locally available
}
else {
    //query from the remote host
}

2.12. Accessing the underlying KafkaStreams object

StreamBuilderFactoryBean from spring-kafka that is responsible for constructing the KafkaStreams object can be accessed programmatically. Each StreamBuilderFactoryBean is registered as stream-builder and appended with the StreamListener method name. If your StreamListener method is named as process for example, the stream builder bean is named as stream-builder-process. Since this is a factory bean, it should be accessed by prepending an ampersand (&) when accessing it programmatically. Following is an example and it assumes the StreamListener method is named as process

StreamsBuilderFactoryBean streamsBuilderFactoryBean = context.getBean("&stream-builder-process", StreamsBuilderFactoryBean.class);
			KafkaStreams kafkaStreams = streamsBuilderFactoryBean.getKafkaStreams();

2.13. State Cleanup

By default, the Kafkastreams.cleanup() method is called when the binding is stopped. See the Spring Kafka documentation. To modify this behavior simply add a single CleanupConfig @Bean (configured to clean up on start, stop, or neither) to the application context; the bean will be detected and wired into the factory bean.

2.14. Health Indicator

The health indicator requires the dependency spring-boot-starter-actuator. For maven use:

<dependency>
  <groupId>org.springframework.boot</groupId>
  <artifactId>spring-boot-starter-actuator</artifactId>
</dependency>

Spring Cloud Stream Binder Kafka Streams provides a health indicator to check the state of the underlying Kafka threads. Spring Cloud Stream defines a property management.health.binders.enabled to enable the health indicator. See the Spring Cloud Stream documentation.

The health indicator provides the following details for each Kafka threads:

  • Thread name

  • Thread state: CREATED, RUNNING, PARTITIONS_REVOKED, PARTITIONS_ASSIGNED, PENDING_SHUTDOWN or DEAD

  • Active tasks: task ID and partitions

  • Standby tasks: task ID and partitions

By default, only the global status is visible (UP or DOWN). To show the details, the property management.endpoint.health.show-details must be set to ALWAYS or WHEN_AUTHORIZED. For more details about the health information, see the Spring Boot Actuator documentation.

The status of the health indicator is UP if all the Kafka threads registered are in the RUNNING state.

2.15. Functional Kafka Streams Applications

Starting 2.2.0.RELEASE, Kafka Streams binder supports the ability to write Kafka Streams applications by simply just implementing the java.util.function.Function or java.util.consumer.Consumer interfaces in Java. In this section, we will see the details of how the functional support work in the binder. The above StreamListener based model can be converted as below.

@SpringBootApplication
@EnableBinding(KafkaStreamsProcessor.class)
public class WordCountProcessorApplication {

	@Bean
	public Function<KStream<?, String>, KStream<?, WordCount>> process() {
		return input ->
		        input
                    .flatMapValues(value -> Arrays.asList(value.toLowerCase().split("\\W+")))
                    .groupBy((key, value) -> value)
                    .windowedBy(TimeWindows.of(5000))
                    .count(Materialized.as("WordCounts-multi"))
                    .toStream()
                    .map((key, value) -> new KeyValue<>(null, new WordCount(key.key(), value, new Date(key.window().start()), new Date(key.window().end()))));
    }

	public static void main(String[] args) {
		SpringApplication.run(WordCountProcessorApplication.class, args);
	}

The input will be received from the input binding defined in the KafkaStreamsProcessor interface and the output will be sent to the output binding. In this case, the input is a stream of String objects and the output is a stream of WordCount objects.

If the processor does not send any data on the outbound, then this becomes a plain Consumer bean as below. In this example, we are simply receiving some data as a stream of String objects (KStream<?, String>) and possibly doing terminal operations with that data without sending any outputs.

	@Bean
	public Consumer<KStream<?, String>> process() {
		return input ->
		        ....
    }

Applications are free to define custom bindings and use that instead of the out of the box KafkaStreamsProcessor interface.

2.15.1. Functions with multiple input bindings

With StreamListener, we define multiple Input bindings and then later on use them as inputs in the method. With the functions approach, we still need to define those bindings in the binding interface. However, it cannot be used in the same way as in a StreamListener method. We use curried functions to represent multiple input destinations in the same processor. For instance, if a function has 2 inputs, the application define 2 partial functions in the function bean method. Lets see some examples.

	@Bean
    public Function<KStream<String, Long>,
                    Function<KTable<String, String>, KStream<String, Long>>> process() {
        return userClicksStream ->
                (userRegionsTable ->
                  (userClicksStream
                        .leftJoin(userRegionsTable, (clicks, region) -> new RegionWithClicks(region == null ?
                                        "UNKNOWN" : region, clicks),
                                Joined.with(Serdes.String(), Serdes.Long(), null))
                        .map((user, regionWithClicks) -> new KeyValue<>(regionWithClicks.getRegion(),
                                regionWithClicks.getClicks()))
                        .groupByKey(Serialized.with(Serdes.String(), Serdes.Long()))
                        .reduce((firstClicks, secondClicks) -> firstClicks + secondClicks)
                        .toStream()));
    }

In the above function bean, there are two inputs and one output. The function that returns from the method takes a KStream as input, but if you look at the output of this function,, that is another function which takes a KTable as its input. The output of this second function is a KStream which becomes the output of the processor. Another way to look at this is like the following:

Function 1: Function<KStream<String, Long> - KStream input; returns the output of "Function 2" Function 2: Function<KTable<String, String>, KStream<String, Long>> - KTable input; returns KStream<String, Long> which becomes the output of the processor.

Both inputs are available as references in the method body and the applications can perform various operations on them. In this example we use function currying on two partial functions. One thing to keep in mind is that the input bindings must follow a natural order of sorting when you have multiple input bindings, otherwise the binder won’t know which binding to bind for the various function inputs. Here is the corresponding binding interface for the above processor.

interface KStreamKTableProcessor {

    @Input("input-1")
    KStream<?, ?> input1();

    @Input("input-2")
    KTable<?, ?> input2();

    @Output("output")
    KStream<?, ?> output();

}

If you look at the 2 inputs, there is a natural sorting order - i.e. input-1 goes to the first partial function input and input-2 goes to the second partial function.

Here is another example that shows multiple inputs with GlobalKTable.

	@Bean
    public Function<KStream<Long, Order>,
            Function<GlobalKTable<Long, Customer>,
                    Function<GlobalKTable<Long, Product>, KStream<Long, EnrichedOrder>>>> process() {

        return orderStream -> (
                customers -> (
                        products -> (
                                orderStream.join(customers,
                                        (orderId, order) -> order.getCustomerId(),
                                        (order, customer) -> new CustomerOrder(customer, order))
                                        .join(products,
                                                (orderId, customerOrder) -> customerOrder
                                                        .productId(),
                                                (customerOrder, product) -> {
                                                    EnrichedOrder enrichedOrder = new EnrichedOrder();
                                                    enrichedOrder.setProduct(product);
                                                    enrichedOrder.setCustomer(customerOrder.customer);
                                                    enrichedOrder.setOrder(customerOrder.order);
                                                    return enrichedOrder;
                                                })
                        )
                )
        );
    }

Here we have 3 inputs. The first function takes a KStream and its output is another Function that takes a GlobalKTable as its input and another function as its output. This last function takes another GlobalKTable as its input and a KStream is provided as this function’s output which will be used as the processor’s output.

Here is a sequential way to conceptualize this:

Function 1: Function<KStream<Long, Order> - KStream input; returns the output of "Function 2" Function 2: Function<GlobalKTable<Long, Customer>, KStream<String, Long>> - GlobalKTable input; returns the output of "Function 3" Function 3: Function<GlobalKTable<Long, Product>, KStream<Long, EnrichedOrder>>> - GlobalKTable input; returns KStream which becomes the output of the processor.

In this example, we have three curried functions. Behind the scenes, the binder will call the apply method on those functions in the order that they appear.

Here is the corresponding binding interface for this application.

interface CustomGlobalKTableProcessor {

        @Input("input-1")
        KStream<?, ?> input1();

        @Input("input-2")
        GlobalKTable<?, ?> input2();

        @Input("input-3")
        GlobalKTable<?, ?> input3();

        @Output("output")
        KStream<?, ?> output();
}

Here also, the input bindings follow a natural order.

2.15.2. Multiple functions in the same application

Multiple functions aan be defined in the same application. When doing this, the binder will do a natural sorting on multiple function bean names first and then apply input and output bindings on them in the natural order. Consider the following two function beans in the same application.

    @Bean
    public Function<KStream<?, String>, KStream<?, WordCount>> process1() {

    }

    @Bean
    public Function<KStream<?, String>, KStream<?, WordCount>> process2() {

    }

Consider also the following binding interface.

interface Bindings {

    @Input("input-1")
    KStream<?, ?> input1();

    @Input("input-2")
    KStream<?, ?> input2();

    @Ouput("output-1")
    KStream<?, ?> output1();

    @Output("output-2")
    KStream<?, ?> output1();
}

Binder will first take the method process1 and use input binding input-1 and output binding output-1. Similarly, for the method process2, it will use input binding input-2 and output binding output-2.

2.15.3. Using custom state stores in functional applications

You can define custom state stores as beans in your application and those will be detected and added to the Kafka Streams builder by the binder. Note that, for regular StreamListener based processors, you still need to use the KafkaStreamsStateStore annotation for custom state stores. Here is an example of using custom state stores with functional style described in this section.

@Bean
		public StoreBuilder myStore() {
			return Stores.keyValueStoreBuilder(
					Stores.persistentKeyValueStore("my-store"), Serdes.Long(),
					Serdes.Long());
		}

		@Bean
		public StoreBuilder otherStore() {
			return Stores.windowStoreBuilder(
					Stores.persistentWindowStore("other-store",
							1L, 3, 3L, false), Serdes.Long(),
					Serdes.Long());
		}

These state stores can be then accessed by the applications directly.

Appendices

Appendix A: Building

A.1. Basic Compile and Test

To build the source you will need to install JDK 1.7.

The build uses the Maven wrapper so you don’t have to install a specific version of Maven. To enable the tests, you should have Kafka server 0.9 or above running before building. See below for more information on running the servers.

The main build command is

$ ./mvnw clean install

You can also add '-DskipTests' if you like, to avoid running the tests.

You can also install Maven (>=3.3.3) yourself and run the mvn command in place of ./mvnw in the examples below. If you do that you also might need to add -P spring if your local Maven settings do not contain repository declarations for spring pre-release artifacts.
Be aware that you might need to increase the amount of memory available to Maven by setting a MAVEN_OPTS environment variable with a value like -Xmx512m -XX:MaxPermSize=128m. We try to cover this in the .mvn configuration, so if you find you have to do it to make a build succeed, please raise a ticket to get the settings added to source control.

The projects that require middleware generally include a docker-compose.yml, so consider using Docker Compose to run the middeware servers in Docker containers.

A.2. Documentation

There is a "full" profile that will generate documentation.

A.3. Working with the code

If you don’t have an IDE preference we would recommend that you use Spring Tools Suite or Eclipse when working with the code. We use the m2eclipe eclipse plugin for maven support. Other IDEs and tools should also work without issue.

A.3.1. Importing into eclipse with m2eclipse

We recommend the m2eclipe eclipse plugin when working with eclipse. If you don’t already have m2eclipse installed it is available from the "eclipse marketplace".

Unfortunately m2e does not yet support Maven 3.3, so once the projects are imported into Eclipse you will also need to tell m2eclipse to use the .settings.xml file for the projects. If you do not do this you may see many different errors related to the POMs in the projects. Open your Eclipse preferences, expand the Maven preferences, and select User Settings. In the User Settings field click Browse and navigate to the Spring Cloud project you imported selecting the .settings.xml file in that project. Click Apply and then OK to save the preference changes.

Alternatively you can copy the repository settings from .settings.xml into your own ~/.m2/settings.xml.

A.3.2. Importing into eclipse without m2eclipse

If you prefer not to use m2eclipse you can generate eclipse project metadata using the following command:

$ ./mvnw eclipse:eclipse

The generated eclipse projects can be imported by selecting import existing projects from the file menu.

[[contributing] == Contributing

Spring Cloud is released under the non-restrictive Apache 2.0 license, and follows a very standard Github development process, using Github tracker for issues and merging pull requests into master. If you want to contribute even something trivial please do not hesitate, but follow the guidelines below.

A.4. Sign the Contributor License Agreement

Before we accept a non-trivial patch or pull request we will need you to sign the contributor’s agreement. Signing the contributor’s agreement does not grant anyone commit rights to the main repository, but it does mean that we can accept your contributions, and you will get an author credit if we do. Active contributors might be asked to join the core team, and given the ability to merge pull requests.

A.5. Code Conventions and Housekeeping

None of these is essential for a pull request, but they will all help. They can also be added after the original pull request but before a merge.

  • Use the Spring Framework code format conventions. If you use Eclipse you can import formatter settings using the eclipse-code-formatter.xml file from the Spring Cloud Build project. If using IntelliJ, you can use the Eclipse Code Formatter Plugin to import the same file.

  • Make sure all new .java files to have a simple Javadoc class comment with at least an @author tag identifying you, and preferably at least a paragraph on what the class is for.

  • Add the ASF license header comment to all new .java files (copy from existing files in the project)

  • Add yourself as an @author to the .java files that you modify substantially (more than cosmetic changes).

  • Add some Javadocs and, if you change the namespace, some XSD doc elements.

  • A few unit tests would help a lot as well — someone has to do it.

  • If no-one else is using your branch, please rebase it against the current master (or other target branch in the main project).

  • When writing a commit message please follow these conventions, if you are fixing an existing issue please add Fixes gh-XXXX at the end of the commit message (where XXXX is the issue number).