diff --git a/docs/_index.md b/docs/_index.md new file mode 100644 index 0000000000000..69a31e04fc570 --- /dev/null +++ b/docs/_index.md @@ -0,0 +1,10 @@ +--- +title: AK 3.7.X +description: Documentation for AK 3.7.X +weight: +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + diff --git a/docs/api.html b/docs/api.html deleted file mode 100644 index 7b74d045520e5..0000000000000 --- a/docs/api.html +++ /dev/null @@ -1,110 +0,0 @@ - - - -
diff --git a/docs/apis/_index.md b/docs/apis/_index.md new file mode 100644 index 0000000000000..00c2b8800a924 --- /dev/null +++ b/docs/apis/_index.md @@ -0,0 +1,99 @@ +--- +title: API +description: +weight: 2 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +Kafka includes five core apis: + + 1. The Producer API allows applications to send streams of data to topics in the Kafka cluster. + 2. The Consumer API allows applications to read streams of data from topics in the Kafka cluster. + 3. The Streams API allows transforming streams of data from input topics to output topics. + 4. The Connect API allows implementing connectors that continually pull from some source system or application into Kafka or push from Kafka into some sink system or application. + 5. The Admin API allows managing and inspecting topics, brokers, and other Kafka objects. +Kafka exposes all its functionality over a language independent protocol which has clients available in many programming languages. However only the Java clients are maintained as part of the main Kafka project, the others are available as independent open source projects. A list of non-Java clients is available [here](https://cwiki.apache.org/confluence/display/KAFKA/Clients). + +# Producer API + +The Producer API allows applications to send streams of data to topics in the Kafka cluster. + +Examples showing how to use the producer are given in the [javadocs](/37/javadoc/index.html?org/apache/kafka/clients/producer/KafkaProducer.html "Kafka 3.7 Javadoc"). + +To use the producer, you can use the following maven dependency: + + + + org.apache.kafka + kafka-clients + 3.7.2 + + +# Consumer API + +The Consumer API allows applications to read streams of data from topics in the Kafka cluster. + +Examples showing how to use the consumer are given in the [javadocs](/37/javadoc/index.html?org/apache/kafka/clients/consumer/KafkaConsumer.html "Kafka 3.7 Javadoc"). + +To use the consumer, you can use the following maven dependency: + + + + org.apache.kafka + kafka-clients + 3.7.2 + + +# Streams API + +The Streams API allows transforming streams of data from input topics to output topics. + +Examples showing how to use this library are given in the [javadocs](/37/javadoc/index.html?org/apache/kafka/streams/KafkaStreams.html "Kafka 3.7 Javadoc") + +Additional documentation on using the Streams API is available [here](/37/streams). + +To use Kafka Streams you can use the following maven dependency: + + + + org.apache.kafka + kafka-streams + 3.7.2 + + +When using Scala you may optionally include the `kafka-streams-scala` library. Additional documentation on using the Kafka Streams DSL for Scala is available [in the developer guide](/37/streams/developer-guide/dsl-api.html#scala-dsl). + +To use Kafka Streams DSL for Scala for Scala 2.13 you can use the following maven dependency: + + + + org.apache.kafka + kafka-streams-scala_2.13 + 3.7.2 + + +# Connect API + +The Connect API allows implementing connectors that continually pull from some source data system into Kafka or push from Kafka into some sink data system. + +Many users of Connect won't need to use this API directly, though, they can use pre-built connectors without needing to write any code. Additional information on using Connect is available [here](/documentation.html#connect). + +Those who want to implement custom connectors can see the [javadoc](/37/javadoc/index.html?org/apache/kafka/connect "Kafka 3.7 Javadoc"). + +# Admin API + +The Admin API supports managing and inspecting topics, brokers, acls, and other Kafka objects. + +To use the Admin API, add the following Maven dependency: + + + + org.apache.kafka + kafka-clients + 3.7.2 + + +For more information about the Admin APIs, see the [javadoc](/37/javadoc/index.html?org/apache/kafka/clients/admin/Admin.html "Kafka 3.7 Javadoc"). diff --git a/docs/configuration.html b/docs/configuration.html deleted file mode 100644 index 7bcb097b94462..0000000000000 --- a/docs/configuration.html +++ /dev/null @@ -1,313 +0,0 @@ - - - - -
diff --git a/docs/configuration/_index.md b/docs/configuration/_index.md new file mode 100644 index 0000000000000..70cf6016c470b --- /dev/null +++ b/docs/configuration/_index.md @@ -0,0 +1,10 @@ +--- +title: Configuration +description: +weight: 3 +tags: ['kafka', 'docs', 'configuration'] +aliases: +keywords: +type: docs +--- + diff --git a/docs/configuration/admin-configs.md b/docs/configuration/admin-configs.md new file mode 100644 index 0000000000000..8c602c16ea98f --- /dev/null +++ b/docs/configuration/admin-configs.md @@ -0,0 +1,11 @@ +--- +title: Admin Configs +description: Admin Configs +weight: 7 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +Below is the configuration of the Kafka Admin client library. {{< include-html file="/static/37/generated/admin_client_config.html" >}} diff --git a/docs/configuration/broker-configs.md b/docs/configuration/broker-configs.md new file mode 100644 index 0000000000000..4d5f7fd8741ab --- /dev/null +++ b/docs/configuration/broker-configs.md @@ -0,0 +1,185 @@ +--- +title: Broker Configs +description: Broker Configs +weight: 1 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +The essential configurations are the following: + + * `broker.id` + * `log.dirs` + * `zookeeper.connect` +Topic-level configurations and defaults are discussed in more detail below. {{< include-html file="/static/37/generated/kafka_config.html" >}} + +More details about broker configuration can be found in the scala class `kafka.server.KafkaConfig`. + +## Updating Broker Configs + +From Kafka version 1.1 onwards, some of the broker configs can be updated without restarting the broker. See the `Dynamic Update Mode` column in Broker Configs for the update mode of each broker config. + + * `read-only`: Requires a broker restart for update + * `per-broker`: May be updated dynamically for each broker + * `cluster-wide`: May be updated dynamically as a cluster-wide default. May also be updated as a per-broker value for testing. + +To alter the current broker configs for broker id 0 (for example, the number of log cleaner threads): + + + > bin/kafka-configs.sh --bootstrap-server localhost:9092 --entity-type brokers --entity-name 0 --alter --add-config log.cleaner.threads=2 + +To describe the current dynamic broker configs for broker id 0: + + + > bin/kafka-configs.sh --bootstrap-server localhost:9092 --entity-type brokers --entity-name 0 --describe + +To delete a config override and revert to the statically configured or default value for broker id 0 (for example, the number of log cleaner threads): + + + > bin/kafka-configs.sh --bootstrap-server localhost:9092 --entity-type brokers --entity-name 0 --alter --delete-config log.cleaner.threads + +Some configs may be configured as a cluster-wide default to maintain consistent values across the whole cluster. All brokers in the cluster will process the cluster default update. For example, to update log cleaner threads on all brokers: + + + > bin/kafka-configs.sh --bootstrap-server localhost:9092 --entity-type brokers --entity-default --alter --add-config log.cleaner.threads=2 + +To describe the currently configured dynamic cluster-wide default configs: + + + > bin/kafka-configs.sh --bootstrap-server localhost:9092 --entity-type brokers --entity-default --describe + +All configs that are configurable at cluster level may also be configured at per-broker level (e.g. for testing). If a config value is defined at different levels, the following order of precedence is used: + + * Dynamic per-broker config stored in ZooKeeper + * Dynamic cluster-wide default config stored in ZooKeeper + * Static broker config from `server.properties` + * Kafka default, see broker configs + + + +### Updating Password Configs Dynamically + +Password config values that are dynamically updated are encrypted before storing in ZooKeeper. The broker config `password.encoder.secret` must be configured in `server.properties` to enable dynamic update of password configs. The secret may be different on different brokers. + +The secret used for password encoding may be rotated with a rolling restart of brokers. The old secret used for encoding passwords currently in ZooKeeper must be provided in the static broker config `password.encoder.old.secret` and the new secret must be provided in `password.encoder.secret`. All dynamic password configs stored in ZooKeeper will be re-encoded with the new secret when the broker starts up. + +In Kafka 1.1.x, all dynamically updated password configs must be provided in every alter request when updating configs using `kafka-configs.sh` even if the password config is not being altered. This constraint will be removed in a future release. + +### Updating Password Configs in ZooKeeper Before Starting Brokers + +From Kafka 2.0.0 onwards, `kafka-configs.sh` enables dynamic broker configs to be updated using ZooKeeper before starting brokers for bootstrapping. This enables all password configs to be stored in encrypted form, avoiding the need for clear passwords in `server.properties`. The broker config `password.encoder.secret` must also be specified if any password configs are included in the alter command. Additional encryption parameters may also be specified. Password encoder configs will not be persisted in ZooKeeper. For example, to store SSL key password for listener `INTERNAL` on broker 0: + + + > bin/kafka-configs.sh --zookeeper localhost:2182 --zk-tls-config-file zk_tls_config.properties --entity-type brokers --entity-name 0 --alter --add-config + 'listener.name.internal.ssl.key.password=key-password,password.encoder.secret=secret,password.encoder.iterations=8192' + +The configuration `listener.name.internal.ssl.key.password` will be persisted in ZooKeeper in encrypted form using the provided encoder configs. The encoder secret and iterations are not persisted in ZooKeeper. + +### Updating SSL Keystore of an Existing Listener + +Brokers may be configured with SSL keystores with short validity periods to reduce the risk of compromised certificates. Keystores may be updated dynamically without restarting the broker. The config name must be prefixed with the listener prefix `listener.name.{listenerName}.` so that only the keystore config of a specific listener is updated. The following configs may be updated in a single alter request at per-broker level: + + * `ssl.keystore.type` + * `ssl.keystore.location` + * `ssl.keystore.password` + * `ssl.key.password` + +If the listener is the inter-broker listener, the update is allowed only if the new keystore is trusted by the truststore configured for that listener. For other listeners, no trust validation is performed on the keystore by the broker. Certificates must be signed by the same certificate authority that signed the old certificate to avoid any client authentication failures. + +### Updating SSL Truststore of an Existing Listener + +Broker truststores may be updated dynamically without restarting the broker to add or remove certificates. Updated truststore will be used to authenticate new client connections. The config name must be prefixed with the listener prefix `listener.name.{listenerName}.` so that only the truststore config of a specific listener is updated. The following configs may be updated in a single alter request at per-broker level: + + * `ssl.truststore.type` + * `ssl.truststore.location` + * `ssl.truststore.password` + +If the listener is the inter-broker listener, the update is allowed only if the existing keystore for that listener is trusted by the new truststore. For other listeners, no trust validation is performed by the broker before the update. Removal of CA certificates used to sign client certificates from the new truststore can lead to client authentication failures. + +### Updating Default Topic Configuration + +Default topic configuration options used by brokers may be updated without broker restart. The configs are applied to topics without a topic config override for the equivalent per-topic config. One or more of these configs may be overridden at cluster-default level used by all brokers. + + * `log.segment.bytes` + * `log.roll.ms` + * `log.roll.hours` + * `log.roll.jitter.ms` + * `log.roll.jitter.hours` + * `log.index.size.max.bytes` + * `log.flush.interval.messages` + * `log.flush.interval.ms` + * `log.retention.bytes` + * `log.retention.ms` + * `log.retention.minutes` + * `log.retention.hours` + * `log.index.interval.bytes` + * `log.cleaner.delete.retention.ms` + * `log.cleaner.min.compaction.lag.ms` + * `log.cleaner.max.compaction.lag.ms` + * `log.cleaner.min.cleanable.ratio` + * `log.cleanup.policy` + * `log.segment.delete.delay.ms` + * `unclean.leader.election.enable` + * `min.insync.replicas` + * `max.message.bytes` + * `compression.type` + * `log.preallocate` + * `log.message.timestamp.type` + * `log.message.timestamp.difference.max.ms` + +From Kafka version 2.0.0 onwards, unclean leader election is automatically enabled by the controller when the config `unclean.leader.election.enable` is dynamically updated. In Kafka version 1.1.x, changes to `unclean.leader.election.enable` take effect only when a new controller is elected. Controller re-election may be forced by running: + + + > bin/zookeeper-shell.sh localhost + rmr /controller + +### Updating Log Cleaner Configs + +Log cleaner configs may be updated dynamically at cluster-default level used by all brokers. The changes take effect on the next iteration of log cleaning. One or more of these configs may be updated: + + * `log.cleaner.threads` + * `log.cleaner.io.max.bytes.per.second` + * `log.cleaner.dedupe.buffer.size` + * `log.cleaner.io.buffer.size` + * `log.cleaner.io.buffer.load.factor` + * `log.cleaner.backoff.ms` + + + +### Updating Thread Configs + +The size of various thread pools used by the broker may be updated dynamically at cluster-default level used by all brokers. Updates are restricted to the range `currentSize / 2` to `currentSize * 2` to ensure that config updates are handled gracefully. + + * `num.network.threads` + * `num.io.threads` + * `num.replica.fetchers` + * `num.recovery.threads.per.data.dir` + * `log.cleaner.threads` + * `background.threads` + + + +### Updating ConnectionQuota Configs + +The maximum number of connections allowed for a given IP/host by the broker may be updated dynamically at cluster-default level used by all brokers. The changes will apply for new connection creations and the existing connections count will be taken into account by the new limits. + + * `max.connections.per.ip` + * `max.connections.per.ip.overrides` + + + +### Adding and Removing Listeners + +Listeners may be added or removed dynamically. When a new listener is added, security configs of the listener must be provided as listener configs with the listener prefix `listener.name.{listenerName}.`. If the new listener uses SASL, the JAAS configuration of the listener must be provided using the JAAS configuration property `sasl.jaas.config` with the listener and mechanism prefix. See JAAS configuration for Kafka brokers for details. + +In Kafka version 1.1.x, the listener used by the inter-broker listener may not be updated dynamically. To update the inter-broker listener to a new listener, the new listener may be added on all brokers without restarting the broker. A rolling restart is then required to update `inter.broker.listener.name`. + +In addition to all the security configs of new listeners, the following configs may be updated dynamically at per-broker level: + + * `listeners` + * `advertised.listeners` + * `listener.security.protocol.map` + +Inter-broker listener must be configured using the static broker configuration `inter.broker.listener.name` or `security.inter.broker.protocol`. diff --git a/docs/configuration/consumer-configs.md b/docs/configuration/consumer-configs.md new file mode 100644 index 0000000000000..24a60ca5abb85 --- /dev/null +++ b/docs/configuration/consumer-configs.md @@ -0,0 +1,11 @@ +--- +title: Consumer Configs +description: Consumer Configs +weight: 4 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +Below is the configuration for the consumer: {{< include-html file="/static/37/generated/consumer_config.html" >}} diff --git a/docs/configuration/kafka-connect-configs.md b/docs/configuration/kafka-connect-configs.md new file mode 100644 index 0000000000000..882c196dcf0ce --- /dev/null +++ b/docs/configuration/kafka-connect-configs.md @@ -0,0 +1,19 @@ +--- +title: Kafka Connect Configs +description: Kafka Connect Configs +weight: 5 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +Below is the configuration of the Kafka Connect framework. {{< include-html file="/static/37/generated/connect_config.html" >}} + +## Source Connector Configs + +Below is the configuration of a source connector. {{< include-html file="/static/37/generated/source_connector_config.html" >}} + +## Sink Connector Configs + +Below is the configuration of a sink connector. {{< include-html file="/static/37/generated/sink_connector_config.html" >}} diff --git a/docs/configuration/kafka-streams-configs.md b/docs/configuration/kafka-streams-configs.md new file mode 100644 index 0000000000000..64c97e60fb6bf --- /dev/null +++ b/docs/configuration/kafka-streams-configs.md @@ -0,0 +1,11 @@ +--- +title: Kafka Streams Configs +description: Kafka Streams Configs +weight: 6 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +Below is the configuration of the Kafka Streams client library. {{< include-html file="/static/37/generated/streams_config.html" >}} diff --git a/docs/configuration/mirrormaker-configs.md b/docs/configuration/mirrormaker-configs.md new file mode 100644 index 0000000000000..85f5406cdfb38 --- /dev/null +++ b/docs/configuration/mirrormaker-configs.md @@ -0,0 +1,27 @@ +--- +title: MirrorMaker Configs +description: MirrorMaker Configs +weight: 8 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +Below is the configuration of the connectors that make up MirrorMaker 2. + +## MirrorMaker Common Configs + +Below are the common configuration properties that apply to all three connectors. {{< include-html file="/static/37/generated/mirror_connector_config.html" >}} + +## MirrorMaker Source Configs + +Below is the configuration of MirrorMaker 2 source connector for replicating topics. {{< include-html file="/static/37/generated/mirror_source_config.html" >}} + +## MirrorMaker Checkpoint Configs + +Below is the configuration of MirrorMaker 2 checkpoint connector for emitting consumer offset checkpoints. {{< include-html file="/static/37/generated/mirror_checkpoint_config.html" >}} + +## MirrorMaker HeartBeat Configs + +Below is the configuration of MirrorMaker 2 heartbeat connector for checking connectivity between connectors and clusters. {{< include-html file="/static/37/generated/mirror_heartbeat_config.html" >}} diff --git a/docs/configuration/producer-configs.md b/docs/configuration/producer-configs.md new file mode 100644 index 0000000000000..0f64cf3e64caa --- /dev/null +++ b/docs/configuration/producer-configs.md @@ -0,0 +1,11 @@ +--- +title: Producer Configs +description: Producer Configs +weight: 3 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +Below is the configuration of the producer: {{< include-html file="/static/37/generated/producer_config.html" >}} diff --git a/docs/configuration/system-properties.md b/docs/configuration/system-properties.md new file mode 100644 index 0000000000000..40b52d75d8831 --- /dev/null +++ b/docs/configuration/system-properties.md @@ -0,0 +1,29 @@ +--- +title: System Properties +description: System Properties +weight: 9 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +Kafka supports some configuration that can be enabled through Java system properties. System properties are usually set by passing the -D flag to the Java virtual machine in which Kafka components are running. Below are the supported system properties. + + * #### org.apache.kafka.disallowed.login.modules + +This system property is used to disable the problematic login modules usage in SASL JAAS configuration. This property accepts comma-separated list of loginModule names. By default **com.sun.security.auth.module.JndiLoginModule** loginModule is disabled. + +If users want to enable JndiLoginModule, users need to explicitly reset the system property like below. We advise the users to validate configurations and only allow trusted JNDI configurations. For more details [CVE-2023-25194](https://nvd.nist.gov/vuln/detail/CVE-2023-25194). + + -Dorg.apache.kafka.disallowed.login.modules= + +To disable more loginModules, update the system property with comma-separated loginModule names. Make sure to explicitly add **JndiLoginModule** module name to the comma-separated list like below. + + -Dorg.apache.kafka.disallowed.login.modules=com.sun.security.auth.module.JndiLoginModule,com.ibm.security.auth.module.LdapLoginModule,com.ibm.security.auth.module.Krb5LoginModule + +Since:| 3.4.0 +---|--- +Default Value:| com.sun.security.auth.module.JndiLoginModule + + diff --git a/docs/configuration/tiered-storage-configs.md b/docs/configuration/tiered-storage-configs.md new file mode 100644 index 0000000000000..a1e70e473d247 --- /dev/null +++ b/docs/configuration/tiered-storage-configs.md @@ -0,0 +1,11 @@ +--- +title: Tiered Storage Configs +description: Tiered Storage Configs +weight: 10 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +Below are the configuration properties for Tiered Storage. {{< include-html file="/static/37/generated/remote_log_manager_config.html" >}} {{< include-html file="/static/37/generated/remote_log_metadata_manager_config.html" >}} diff --git a/docs/configuration/topic-level-configs.md b/docs/configuration/topic-level-configs.md new file mode 100644 index 0000000000000..3c0004e0ddc89 --- /dev/null +++ b/docs/configuration/topic-level-configs.md @@ -0,0 +1,34 @@ +--- +title: Topic-Level Configs +description: Topic-Level Configs +weight: 2 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +Configurations pertinent to topics have both a server default as well an optional per-topic override. If no per-topic configuration is given the server default is used. The override can be set at topic creation time by giving one or more `--config` options. This example creates a topic named _my-topic_ with a custom max message size and flush rate: + + + > bin/kafka-topics.sh --bootstrap-server localhost:9092 --create --topic my-topic --partitions 1 \ + --replication-factor 1 --config max.message.bytes=64000 --config flush.messages=1 + +Overrides can also be changed or set later using the alter configs command. This example updates the max message size for _my-topic_ : + + + > bin/kafka-configs.sh --bootstrap-server localhost:9092 --entity-type topics --entity-name my-topic + --alter --add-config max.message.bytes=128000 + +To check overrides set on the topic you can do + + + > bin/kafka-configs.sh --bootstrap-server localhost:9092 --entity-type topics --entity-name my-topic --describe + +To remove an override you can do + + + > bin/kafka-configs.sh --bootstrap-server localhost:9092 --entity-type topics --entity-name my-topic + --alter --delete-config max.message.bytes + +The following are the topic-level configurations. The server's default configuration for this property is given under the Server Default Property heading. A given server default config value only applies to a topic if it does not have an explicit topic config override. {{< include-html file="/static/37/generated/topic_config.html" >}} diff --git a/docs/connect.html b/docs/connect.html deleted file mode 100644 index cf4ff57c29ce1..0000000000000 --- a/docs/connect.html +++ /dev/null @@ -1,1091 +0,0 @@ - - - - -
diff --git a/docs/design.html b/docs/design.html deleted file mode 100644 index 18f78d044c436..0000000000000 --- a/docs/design.html +++ /dev/null @@ -1,681 +0,0 @@ - - - - -
diff --git a/docs/design/_index.md b/docs/design/_index.md new file mode 100644 index 0000000000000..45615eb0613bc --- /dev/null +++ b/docs/design/_index.md @@ -0,0 +1,10 @@ +--- +title: Design +description: +weight: 4 +tags: ['kafka', 'docs', 'design'] +aliases: +keywords: +type: docs +--- + diff --git a/docs/design/design.md b/docs/design/design.md new file mode 100644 index 0000000000000..1bbbd400f9cca --- /dev/null +++ b/docs/design/design.md @@ -0,0 +1,416 @@ +--- +title: Design +description: +weight: 1 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +## Motivation + +We designed Kafka to be able to act as a unified platform for handling all the real-time data feeds a large company might have. To do this we had to think through a fairly broad set of use cases. + +It would have to have high-throughput to support high volume event streams such as real-time log aggregation. + +It would need to deal gracefully with large data backlogs to be able to support periodic data loads from offline systems. + +It also meant the system would have to handle low-latency delivery to handle more traditional messaging use-cases. + +We wanted to support partitioned, distributed, real-time processing of these feeds to create new, derived feeds. This motivated our partitioning and consumer model. + +Finally in cases where the stream is fed into other data systems for serving, we knew the system would have to be able to guarantee fault-tolerance in the presence of machine failures. + +Supporting these uses led us to a design with a number of unique elements, more akin to a database log than a traditional messaging system. We will outline some elements of the design in the following sections. + +## Persistence + +### Don't fear the filesystem! + +Kafka relies heavily on the filesystem for storing and caching messages. There is a general perception that "disks are slow" which makes people skeptical that a persistent structure can offer competitive performance. In fact disks are both much slower and much faster than people expect depending on how they are used; and a properly designed disk structure can often be as fast as the network. + +The key fact about disk performance is that the throughput of hard drives has been diverging from the latency of a disk seek for the last decade. As a result the performance of linear writes on a [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures) configuration with six 7200rpm SATA RAID-5 array is about 600MB/sec but the performance of random writes is only about 100k/sec--a difference of over 6000X. These linear reads and writes are the most predictable of all usage patterns, and are heavily optimized by the operating system. A modern operating system provides read-ahead and write-behind techniques that prefetch data in large block multiples and group smaller logical writes into large physical writes. A further discussion of this issue can be found in this [ACM Queue article](https://queue.acm.org/detail.cfm?id=1563874); they actually find that [sequential disk access can in some cases be faster than random memory access!](https://deliveryimages.acm.org/10.1145/1570000/1563874/jacobs3.jpg) + +To compensate for this performance divergence, modern operating systems have become increasingly aggressive in their use of main memory for disk caching. A modern OS will happily divert _all_ free memory to disk caching with little performance penalty when the memory is reclaimed. All disk reads and writes will go through this unified cache. This feature cannot easily be turned off without using direct I/O, so even if a process maintains an in-process cache of the data, this data will likely be duplicated in OS pagecache, effectively storing everything twice. + +Furthermore, we are building on top of the JVM, and anyone who has spent any time with Java memory usage knows two things: + + 1. The memory overhead of objects is very high, often doubling the size of the data stored (or worse). + 2. Java garbage collection becomes increasingly fiddly and slow as the in-heap data increases. + + + +As a result of these factors using the filesystem and relying on pagecache is superior to maintaining an in-memory cache or other structure--we at least double the available cache by having automatic access to all free memory, and likely double again by storing a compact byte structure rather than individual objects. Doing so will result in a cache of up to 28-30GB on a 32GB machine without GC penalties. Furthermore, this cache will stay warm even if the service is restarted, whereas the in-process cache will need to be rebuilt in memory (which for a 10GB cache may take 10 minutes) or else it will need to start with a completely cold cache (which likely means terrible initial performance). This also greatly simplifies the code as all logic for maintaining coherency between the cache and filesystem is now in the OS, which tends to do so more efficiently and more correctly than one-off in-process attempts. If your disk usage favors linear reads then read-ahead is effectively pre-populating this cache with useful data on each disk read. + +This suggests a design which is very simple: rather than maintain as much as possible in-memory and flush it all out to the filesystem in a panic when we run out of space, we invert that. All data is immediately written to a persistent log on the filesystem without necessarily flushing to disk. In effect this just means that it is transferred into the kernel's pagecache. + +This style of pagecache-centric design is described in an [article](https://varnish-cache.org/wiki/ArchitectNotes) on the design of Varnish here (along with a healthy dose of arrogance). + +### Constant Time Suffices + +The persistent data structure used in messaging systems are often a per-consumer queue with an associated BTree or other general-purpose random access data structures to maintain metadata about messages. BTrees are the most versatile data structure available, and make it possible to support a wide variety of transactional and non-transactional semantics in the messaging system. They do come with a fairly high cost, though: Btree operations are O(log N). Normally O(log N) is considered essentially equivalent to constant time, but this is not true for disk operations. Disk seeks come at 10 ms a pop, and each disk can do only one seek at a time so parallelism is limited. Hence even a handful of disk seeks leads to very high overhead. Since storage systems mix very fast cached operations with very slow physical disk operations, the observed performance of tree structures is often superlinear as data increases with fixed cache--i.e. doubling your data makes things much worse than twice as slow. + +Intuitively a persistent queue could be built on simple reads and appends to files as is commonly the case with logging solutions. This structure has the advantage that all operations are O(1) and reads do not block writes or each other. This has obvious performance advantages since the performance is completely decoupled from the data size--one server can now take full advantage of a number of cheap, low-rotational speed 1+TB SATA drives. Though they have poor seek performance, these drives have acceptable performance for large reads and writes and come at 1/3 the price and 3x the capacity. + +Having access to virtually unlimited disk space without any performance penalty means that we can provide some features not usually found in a messaging system. For example, in Kafka, instead of attempting to delete messages as soon as they are consumed, we can retain messages for a relatively long period (say a week). This leads to a great deal of flexibility for consumers, as we will describe. + +## Efficiency + +We have put significant effort into efficiency. One of our primary use cases is handling web activity data, which is very high volume: each page view may generate dozens of writes. Furthermore, we assume each message published is read by at least one consumer (often many), hence we strive to make consumption as cheap as possible. + +We have also found, from experience building and running a number of similar systems, that efficiency is a key to effective multi-tenant operations. If the downstream infrastructure service can easily become a bottleneck due to a small bump in usage by the application, such small changes will often create problems. By being very fast we help ensure that the application will tip-over under load before the infrastructure. This is particularly important when trying to run a centralized service that supports dozens or hundreds of applications on a centralized cluster as changes in usage patterns are a near-daily occurrence. + +We discussed disk efficiency in the previous section. Once poor disk access patterns have been eliminated, there are two common causes of inefficiency in this type of system: too many small I/O operations, and excessive byte copying. + +The small I/O problem happens both between the client and the server and in the server's own persistent operations. + +To avoid this, our protocol is built around a "message set" abstraction that naturally groups messages together. This allows network requests to group messages together and amortize the overhead of the network roundtrip rather than sending a single message at a time. The server in turn appends chunks of messages to its log in one go, and the consumer fetches large linear chunks at a time. + +This simple optimization produces orders of magnitude speed up. Batching leads to larger network packets, larger sequential disk operations, contiguous memory blocks, and so on, all of which allows Kafka to turn a bursty stream of random message writes into linear writes that flow to the consumers. + +The other inefficiency is in byte copying. At low message rates this is not an issue, but under load the impact is significant. To avoid this we employ a standardized binary message format that is shared by the producer, the broker, and the consumer (so data chunks can be transferred without modification between them). + +The message log maintained by the broker is itself just a directory of files, each populated by a sequence of message sets that have been written to disk in the same format used by the producer and consumer. Maintaining this common format allows optimization of the most important operation: network transfer of persistent log chunks. Modern unix operating systems offer a highly optimized code path for transferring data out of pagecache to a socket; in Linux this is done with the [sendfile system call](https://man7.org/linux/man-pages/man2/sendfile.2.html). + +To understand the impact of sendfile, it is important to understand the common data path for transfer of data from file to socket: + + 1. The operating system reads data from the disk into pagecache in kernel space + 2. The application reads the data from kernel space into a user-space buffer + 3. The application writes the data back into kernel space into a socket buffer + 4. The operating system copies the data from the socket buffer to the NIC buffer where it is sent over the network + + + +This is clearly inefficient, there are four copies and two system calls. Using sendfile, this re-copying is avoided by allowing the OS to send the data from pagecache to the network directly. So in this optimized path, only the final copy to the NIC buffer is needed. + +We expect a common use case to be multiple consumers on a topic. Using the zero-copy optimization above, data is copied into pagecache exactly once and reused on each consumption instead of being stored in memory and copied out to user-space every time it is read. This allows messages to be consumed at a rate that approaches the limit of the network connection. + +This combination of pagecache and sendfile means that on a Kafka cluster where the consumers are mostly caught up you will see no read activity on the disks whatsoever as they will be serving data entirely from cache. + +TLS/SSL libraries operate at the user space (in-kernel `SSL_sendfile` is currently not supported by Kafka). Due to this restriction, `sendfile` is not used when SSL is enabled. For enabling SSL configuration, refer to `security.protocol` and `security.inter.broker.protocol` + +For more background on the sendfile and zero-copy support in Java, see this [article](https://developer.ibm.com/articles/j-zerocopy/). + +### End-to-end Batch Compression + +In some cases the bottleneck is actually not CPU or disk but network bandwidth. This is particularly true for a data pipeline that needs to send messages between data centers over a wide-area network. Of course, the user can always compress its messages one at a time without any support needed from Kafka, but this can lead to very poor compression ratios as much of the redundancy is due to repetition between messages of the same type (e.g. field names in JSON or user agents in web logs or common string values). Efficient compression requires compressing multiple messages together rather than compressing each message individually. + +Kafka supports this with an efficient batching format. A batch of messages can be grouped together, compressed, and sent to the server in this form. The broker decompresses the batch in order to validate it. For example, it validates that the number of records in the batch is same as what batch header states. This batch of messages is then written to disk in compressed form. The batch will remain compressed in the log and it will also be transmitted to the consumer in compressed form. The consumer decompresses any compressed data that it receives. + +Kafka supports GZIP, Snappy, LZ4 and ZStandard compression protocols. More details on compression can be found [here](https://cwiki.apache.org/confluence/display/KAFKA/Compression). + +## The Producer + +### Load balancing + +The producer sends data directly to the broker that is the leader for the partition without any intervening routing tier. To help the producer do this all Kafka nodes can answer a request for metadata about which servers are alive and where the leaders for the partitions of a topic are at any given time to allow the producer to appropriately direct its requests. + +The client controls which partition it publishes messages to. This can be done at random, implementing a kind of random load balancing, or it can be done by some semantic partitioning function. We expose the interface for semantic partitioning by allowing the user to specify a key to partition by and using this to hash to a partition (there is also an option to override the partition function if need be). For example if the key chosen was a user id then all data for a given user would be sent to the same partition. This in turn will allow consumers to make locality assumptions about their consumption. This style of partitioning is explicitly designed to allow locality-sensitive processing in consumers. + +### Asynchronous send + +Batching is one of the big drivers of efficiency, and to enable batching the Kafka producer will attempt to accumulate data in memory and to send out larger batches in a single request. The batching can be configured to accumulate no more than a fixed number of messages and to wait no longer than some fixed latency bound (say 64k or 10 ms). This allows the accumulation of more bytes to send, and few larger I/O operations on the servers. This buffering is configurable and gives a mechanism to trade off a small amount of additional latency for better throughput. + +Details on configuration and the [api](/https://kafka.apache.org/082/javadoc/index.html?org/apache/kafka/clients/producer/KafkaProducer.html) for the producer can be found elsewhere in the documentation. + +## The Consumer + +The Kafka consumer works by issuing "fetch" requests to the brokers leading the partitions it wants to consume. The consumer specifies its offset in the log with each request and receives back a chunk of log beginning from that position. The consumer thus has significant control over this position and can rewind it to re-consume data if need be. + +### Push vs. pull + +An initial question we considered is whether consumers should pull data from brokers or brokers should push data to the consumer. In this respect Kafka follows a more traditional design, shared by most messaging systems, where data is pushed to the broker from the producer and pulled from the broker by the consumer. Some logging-centric systems, such as [Scribe](https://github.com/facebook/scribe) and [Apache Flume](https://flume.apache.org/), follow a very different push-based path where data is pushed downstream. There are pros and cons to both approaches. However, a push-based system has difficulty dealing with diverse consumers as the broker controls the rate at which data is transferred. The goal is generally for the consumer to be able to consume at the maximum possible rate; unfortunately, in a push system this means the consumer tends to be overwhelmed when its rate of consumption falls below the rate of production (a denial of service attack, in essence). A pull-based system has the nicer property that the consumer simply falls behind and catches up when it can. This can be mitigated with some kind of backoff protocol by which the consumer can indicate it is overwhelmed, but getting the rate of transfer to fully utilize (but never over-utilize) the consumer is trickier than it seems. Previous attempts at building systems in this fashion led us to go with a more traditional pull model. + +Another advantage of a pull-based system is that it lends itself to aggressive batching of data sent to the consumer. A push-based system must choose to either send a request immediately or accumulate more data and then send it later without knowledge of whether the downstream consumer will be able to immediately process it. If tuned for low latency, this will result in sending a single message at a time only for the transfer to end up being buffered anyway, which is wasteful. A pull-based design fixes this as the consumer always pulls all available messages after its current position in the log (or up to some configurable max size). So one gets optimal batching without introducing unnecessary latency. + +The deficiency of a naive pull-based system is that if the broker has no data the consumer may end up polling in a tight loop, effectively busy-waiting for data to arrive. To avoid this we have parameters in our pull request that allow the consumer request to block in a "long poll" waiting until data arrives (and optionally waiting until a given number of bytes is available to ensure large transfer sizes). + +You could imagine other possible designs which would be only pull, end-to-end. The producer would locally write to a local log, and brokers would pull from that with consumers pulling from them. A similar type of "store-and-forward" producer is often proposed. This is intriguing but we felt not very suitable for our target use cases which have thousands of producers. Our experience running persistent data systems at scale led us to feel that involving thousands of disks in the system across many applications would not actually make things more reliable and would be a nightmare to operate. And in practice we have found that we can run a pipeline with strong SLAs at large scale without a need for producer persistence. + +### Consumer Position + +Keeping track of _what_ has been consumed is, surprisingly, one of the key performance points of a messaging system. + +Most messaging systems keep metadata about what messages have been consumed on the broker. That is, as a message is handed out to a consumer, the broker either records that fact locally immediately or it may wait for acknowledgement from the consumer. This is a fairly intuitive choice, and indeed for a single machine server it is not clear where else this state could go. Since the data structures used for storage in many messaging systems scale poorly, this is also a pragmatic choice--since the broker knows what is consumed it can immediately delete it, keeping the data size small. + +What is perhaps not obvious is that getting the broker and consumer to come into agreement about what has been consumed is not a trivial problem. If the broker records a message as **consumed** immediately every time it is handed out over the network, then if the consumer fails to process the message (say because it crashes or the request times out or whatever) that message will be lost. To solve this problem, many messaging systems add an acknowledgement feature which means that messages are only marked as **sent** not **consumed** when they are sent; the broker waits for a specific acknowledgement from the consumer to record the message as **consumed**. This strategy fixes the problem of losing messages, but creates new problems. First of all, if the consumer processes the message but fails before it can send an acknowledgement then the message will be consumed twice. The second problem is around performance, now the broker must keep multiple states about every single message (first to lock it so it is not given out a second time, and then to mark it as permanently consumed so that it can be removed). Tricky problems must be dealt with, like what to do with messages that are sent but never acknowledged. + +Kafka handles this differently. Our topic is divided into a set of totally ordered partitions, each of which is consumed by exactly one consumer within each subscribing consumer group at any given time. This means that the position of a consumer in each partition is just a single integer, the offset of the next message to consume. This makes the state about what has been consumed very small, just one number for each partition. This state can be periodically checkpointed. This makes the equivalent of message acknowledgements very cheap. + +There is a side benefit of this decision. A consumer can deliberately _rewind_ back to an old offset and re-consume data. This violates the common contract of a queue, but turns out to be an essential feature for many consumers. For example, if the consumer code has a bug and is discovered after some messages are consumed, the consumer can re-consume those messages once the bug is fixed. + +### Offline Data Load + +Scalable persistence allows for the possibility of consumers that only periodically consume such as batch data loads that periodically bulk-load data into an offline system such as Hadoop or a relational data warehouse. + +In the case of Hadoop we parallelize the data load by splitting the load over individual map tasks, one for each node/topic/partition combination, allowing full parallelism in the loading. Hadoop provides the task management, and tasks which fail can restart without danger of duplicate data--they simply restart from their original position. + +### Static Membership + +Static membership aims to improve the availability of stream applications, consumer groups and other applications built on top of the group rebalance protocol. The rebalance protocol relies on the group coordinator to allocate entity ids to group members. These generated ids are ephemeral and will change when members restart and rejoin. For consumer based apps, this "dynamic membership" can cause a large percentage of tasks re-assigned to different instances during administrative operations such as code deploys, configuration updates and periodic restarts. For large state applications, shuffled tasks need a long time to recover their local states before processing and cause applications to be partially or entirely unavailable. Motivated by this observation, Kafka’s group management protocol allows group members to provide persistent entity ids. Group membership remains unchanged based on those ids, thus no rebalance will be triggered. + +If you want to use static membership, + + * Upgrade both broker cluster and client apps to 2.3 or beyond, and also make sure the upgraded brokers are using `inter.broker.protocol.version` of 2.3 or beyond as well. + * Set the config `ConsumerConfig#GROUP_INSTANCE_ID_CONFIG` to a unique value for each consumer instance under one group. + * For Kafka Streams applications, it is sufficient to set a unique `ConsumerConfig#GROUP_INSTANCE_ID_CONFIG` per KafkaStreams instance, independent of the number of used threads for an instance. + +If your broker is on an older version than 2.3, but you choose to set `ConsumerConfig#GROUP_INSTANCE_ID_CONFIG` on the client side, the application will detect the broker version and then throws an UnsupportedException. If you accidentally configure duplicate ids for different instances, a fencing mechanism on broker side will inform your duplicate client to shutdown immediately by triggering a `org.apache.kafka.common.errors.FencedInstanceIdException`. For more details, see [KIP-345](https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances) + +## Message Delivery Semantics + +Now that we understand a little about how producers and consumers work, let's discuss the semantic guarantees Kafka provides between producer and consumer. Clearly there are multiple possible message delivery guarantees that could be provided: + + * _At most once_ --Messages may be lost but are never redelivered. + * _At least once_ --Messages are never lost but may be redelivered. + * _Exactly once_ --this is what people actually want, each message is delivered once and only once. + +It's worth noting that this breaks down into two problems: the durability guarantees for publishing a message and the guarantees when consuming a message. + +Many systems claim to provide "exactly once" delivery semantics, but it is important to read the fine print, most of these claims are misleading (i.e. they don't translate to the case where consumers or producers can fail, cases where there are multiple consumer processes, or cases where data written to disk can be lost). + +Kafka's semantics are straight-forward. When publishing a message we have a notion of the message being "committed" to the log. Once a published message is committed it will not be lost as long as one broker that replicates the partition to which this message was written remains "alive". The definition of committed message, alive partition as well as a description of which types of failures we attempt to handle will be described in more detail in the next section. For now let's assume a perfect, lossless broker and try to understand the guarantees to the producer and consumer. If a producer attempts to publish a message and experiences a network error it cannot be sure if this error happened before or after the message was committed. This is similar to the semantics of inserting into a database table with an autogenerated key. + +Prior to 0.11.0.0, if a producer failed to receive a response indicating that a message was committed, it had little choice but to resend the message. This provides at-least-once delivery semantics since the message may be written to the log again during resending if the original request had in fact succeeded. Since 0.11.0.0, the Kafka producer also supports an idempotent delivery option which guarantees that resending will not result in duplicate entries in the log. To achieve this, the broker assigns each producer an ID and deduplicates messages using a sequence number that is sent by the producer along with every message. Also beginning with 0.11.0.0, the producer supports the ability to send messages to multiple topic partitions using transaction-like semantics: i.e. either all messages are successfully written or none of them are. The main use case for this is exactly-once processing between Kafka topics (described below). + +Not all use cases require such strong guarantees. For uses which are latency sensitive we allow the producer to specify the durability level it desires. If the producer specifies that it wants to wait on the message being committed this can take on the order of 10 ms. However the producer can also specify that it wants to perform the send completely asynchronously or that it wants to wait only until the leader (but not necessarily the followers) have the message. + +Now let's describe the semantics from the point-of-view of the consumer. All replicas have the exact same log with the same offsets. The consumer controls its position in this log. If the consumer never crashed it could just store this position in memory, but if the consumer fails and we want this topic partition to be taken over by another process the new process will need to choose an appropriate position from which to start processing. Let's say the consumer reads some messages -- it has several options for processing the messages and updating its position. + + 1. It can read the messages, then save its position in the log, and finally process the messages. In this case there is a possibility that the consumer process crashes after saving its position but before saving the output of its message processing. In this case the process that took over processing would start at the saved position even though a few messages prior to that position had not been processed. This corresponds to "at-most-once" semantics as in the case of a consumer failure messages may not be processed. + 2. It can read the messages, process the messages, and finally save its position. In this case there is a possibility that the consumer process crashes after processing messages but before saving its position. In this case when the new process takes over the first few messages it receives will already have been processed. This corresponds to the "at-least-once" semantics in the case of consumer failure. In many cases messages have a primary key and so the updates are idempotent (receiving the same message twice just overwrites a record with another copy of itself). + + +So what about exactly once semantics (i.e. the thing you actually want)? When consuming from a Kafka topic and producing to another topic (as in a [Kafka Streams](https://kafka.apache.org/streams) application), we can leverage the new transactional producer capabilities in 0.11.0.0 that were mentioned above. The consumer's position is stored as a message in a topic, so we can write the offset to Kafka in the same transaction as the output topics receiving the processed data. If the transaction is aborted, the consumer's position will revert to its old value and the produced data on the output topics will not be visible to other consumers, depending on their "isolation level." In the default "read_uncommitted" isolation level, all messages are visible to consumers even if they were part of an aborted transaction, but in "read_committed," the consumer will only return messages from transactions which were committed (and any messages which were not part of a transaction). + +When writing to an external system, the limitation is in the need to coordinate the consumer's position with what is actually stored as output. The classic way of achieving this would be to introduce a two-phase commit between the storage of the consumer position and the storage of the consumers output. But this can be handled more simply and generally by letting the consumer store its offset in the same place as its output. This is better because many of the output systems a consumer might want to write to will not support a two-phase commit. As an example of this, consider a [Kafka Connect](https://kafka.apache.org/#connect) connector which populates data in HDFS along with the offsets of the data it reads so that it is guaranteed that either data and offsets are both updated or neither is. We follow similar patterns for many other data systems which require these stronger semantics and for which the messages do not have a primary key to allow for deduplication. + +So effectively Kafka supports exactly-once delivery in [Kafka Streams](https://kafka.apache.org/streams), and the transactional producer/consumer can be used generally to provide exactly-once delivery when transferring and processing data between Kafka topics. Exactly-once delivery for other destination systems generally requires cooperation with such systems, but Kafka provides the offset which makes implementing this feasible (see also [Kafka Connect](https://kafka.apache.org/#connect)). Otherwise, Kafka guarantees at-least-once delivery by default, and allows the user to implement at-most-once delivery by disabling retries on the producer and committing offsets in the consumer prior to processing a batch of messages. + +## Replication + +Kafka replicates the log for each topic's partitions across a configurable number of servers (you can set this replication factor on a topic-by-topic basis). This allows automatic failover to these replicas when a server in the cluster fails so messages remain available in the presence of failures. + +Other messaging systems provide some replication-related features, but, in our (totally biased) opinion, this appears to be a tacked-on thing, not heavily used, and with large downsides: replicas are inactive, throughput is heavily impacted, it requires fiddly manual configuration, etc. Kafka is meant to be used with replication by default--in fact we implement un-replicated topics as replicated topics where the replication factor is one. + +The unit of replication is the topic partition. Under non-failure conditions, each partition in Kafka has a single leader and zero or more followers. The total number of replicas including the leader constitute the replication factor. All writes go to the leader of the partition, and reads can go to the leader or the followers of the partition. Typically, there are many more partitions than brokers and the leaders are evenly distributed among brokers. The logs on the followers are identical to the leader's log--all have the same offsets and messages in the same order (though, of course, at any given time the leader may have a few as-yet unreplicated messages at the end of its log). + +Followers consume messages from the leader just as a normal Kafka consumer would and apply them to their own log. Having the followers pull from the leader has the nice property of allowing the follower to naturally batch together log entries they are applying to their log. + +As with most distributed systems, automatically handling failures requires a precise definition of what it means for a node to be "alive." In Kafka, a special node known as the "controller" is responsible for managing the registration of brokers in the cluster. Broker liveness has two conditions: + + 1. Brokers must maintain an active session with the controller in order to receive regular metadata updates. + 2. Brokers acting as followers must replicate the writes from the leader and not fall "too far" behind. + + + +What is meant by an "active session" depends on the cluster configuration. For KRaft clusters, an active session is maintained by sending periodic heartbeats to the controller. If the controller fails to receive a heartbeat before the timeout configured by `broker.session.timeout.ms` expires, then the node is considered offline. + +For clusters using Zookeeper, liveness is determined indirectly through the existence of an ephemeral node which is created by the broker on initialization of its Zookeeper session. If the broker loses its session after failing to send heartbeats to Zookeeper before expiration of `zookeeper.session.timeout.ms`, then the node gets deleted. The controller would then notice the node deletion through a Zookeeper watch and mark the broker offline. + +We refer to nodes satisfying these two conditions as being "in sync" to avoid the vagueness of "alive" or "failed". The leader keeps track of the set of "in sync" replicas, which is known as the ISR. If either of these conditions fail to be satisfied, then the broker will be removed from the ISR. For example, if a follower dies, then the controller will notice the failure through the loss of its session, and will remove the broker from the ISR. On the other hand, if the follower lags too far behind the leader but still has an active session, then the leader can also remove it from the ISR. The determination of lagging replicas is controlled through the `replica.lag.time.max.ms` configuration. Replicas that cannot catch up to the end of the log on the leader within the max time set by this configuration are removed from the ISR. + +In distributed systems terminology we only attempt to handle a "fail/recover" model of failures where nodes suddenly cease working and then later recover (perhaps without knowing that they have died). Kafka does not handle so-called "Byzantine" failures in which nodes produce arbitrary or malicious responses (perhaps due to bugs or foul play). + +We can now more precisely define that a message is considered committed when all replicas in the ISR for that partition have applied it to their log. Only committed messages are ever given out to the consumer. This means that the consumer need not worry about potentially seeing a message that could be lost if the leader fails. Producers, on the other hand, have the option of either waiting for the message to be committed or not, depending on their preference for tradeoff between latency and durability. This preference is controlled by the acks setting that the producer uses. Note that topics have a setting for the "minimum number" of in-sync replicas that is checked when the producer requests acknowledgment that a message has been written to the full set of in-sync replicas. If a less stringent acknowledgement is requested by the producer, then the message can be committed, and consumed, even if the number of in-sync replicas is lower than the minimum (e.g. it can be as low as just the leader). + +The guarantee that Kafka offers is that a committed message will not be lost, as long as there is at least one in sync replica alive, at all times. + +Kafka will remain available in the presence of node failures after a short fail-over period, but may not remain available in the presence of network partitions. + +### Replicated Logs: Quorums, ISRs, and State Machines (Oh my!) + +At its heart a Kafka partition is a replicated log. The replicated log is one of the most basic primitives in distributed data systems, and there are many approaches for implementing one. A replicated log can be used by other systems as a primitive for implementing other distributed systems in the [state-machine style](https://en.wikipedia.org/wiki/State_machine_replication). + +A replicated log models the process of coming into consensus on the order of a series of values (generally numbering the log entries 0, 1, 2, ...). There are many ways to implement this, but the simplest and fastest is with a leader who chooses the ordering of values provided to it. As long as the leader remains alive, all followers need to only copy the values and ordering the leader chooses. + +Of course if leaders didn't fail we wouldn't need followers! When the leader does die we need to choose a new leader from among the followers. But followers themselves may fall behind or crash so we must ensure we choose an up-to-date follower. The fundamental guarantee a log replication algorithm must provide is that if we tell the client a message is committed, and the leader fails, the new leader we elect must also have that message. This yields a tradeoff: if the leader waits for more followers to acknowledge a message before declaring it committed then there will be more potentially electable leaders. + +If you choose the number of acknowledgements required and the number of logs that must be compared to elect a leader such that there is guaranteed to be an overlap, then this is called a Quorum. + +A common approach to this tradeoff is to use a majority vote for both the commit decision and the leader election. This is not what Kafka does, but let's explore it anyway to understand the tradeoffs. Let's say we have 2 _f_ +1 replicas. If _f_ +1 replicas must receive a message prior to a commit being declared by the leader, and if we elect a new leader by electing the follower with the most complete log from at least _f_ +1 replicas, then, with no more than _f_ failures, the leader is guaranteed to have all committed messages. This is because among any _f_ +1 replicas, there must be at least one replica that contains all committed messages. That replica's log will be the most complete and therefore will be selected as the new leader. There are many remaining details that each algorithm must handle (such as precisely defined what makes a log more complete, ensuring log consistency during leader failure or changing the set of servers in the replica set) but we will ignore these for now. + +This majority vote approach has a very nice property: the latency is dependent on only the fastest servers. That is, if the replication factor is three, the latency is determined by the faster follower not the slower one. + +There are a rich variety of algorithms in this family including ZooKeeper's [Zab](https://web.archive.org/web/20140602093727/https://www.stanford.edu/class/cs347/reading/zab.pdf), [Raft](https://www.usenix.org/system/files/conference/atc14/atc14-paper-ongaro.pdf), and [Viewstamped Replication](https://pmg.csail.mit.edu/papers/vr-revisited.pdf). The most similar academic publication we are aware of to Kafka's actual implementation is [PacificA](https://research.microsoft.com/apps/pubs/default.aspx?id=66814) from Microsoft. + +The downside of majority vote is that it doesn't take many failures to leave you with no electable leaders. To tolerate one failure requires three copies of the data, and to tolerate two failures requires five copies of the data. In our experience having only enough redundancy to tolerate a single failure is not enough for a practical system, but doing every write five times, with 5x the disk space requirements and 1/5th the throughput, is not very practical for large volume data problems. This is likely why quorum algorithms more commonly appear for shared cluster configuration such as ZooKeeper but are less common for primary data storage. For example in HDFS the namenode's high-availability feature is built on a [majority-vote-based journal](https://blog.cloudera.com/blog/2012/10/quorum-based-journaling-in-cdh4-1), but this more expensive approach is not used for the data itself. + +Kafka takes a slightly different approach to choosing its quorum set. Instead of majority vote, Kafka dynamically maintains a set of in-sync replicas (ISR) that are caught-up to the leader. Only members of this set are eligible for election as leader. A write to a Kafka partition is not considered committed until _all_ in-sync replicas have received the write. This ISR set is persisted in the cluster metadata whenever it changes. Because of this, any replica in the ISR is eligible to be elected leader. This is an important factor for Kafka's usage model where there are many partitions and ensuring leadership balance is important. With this ISR model and _f+1_ replicas, a Kafka topic can tolerate _f_ failures without losing committed messages. + +For most use cases we hope to handle, we think this tradeoff is a reasonable one. In practice, to tolerate _f_ failures, both the majority vote and the ISR approach will wait for the same number of replicas to acknowledge before committing a message (e.g. to survive one failure a majority quorum needs three replicas and one acknowledgement and the ISR approach requires two replicas and one acknowledgement). The ability to commit without the slowest servers is an advantage of the majority vote approach. However, we think it is ameliorated by allowing the client to choose whether they block on the message commit or not, and the additional throughput and disk space due to the lower required replication factor is worth it. + +Another important design distinction is that Kafka does not require that crashed nodes recover with all their data intact. It is not uncommon for replication algorithms in this space to depend on the existence of "stable storage" that cannot be lost in any failure-recovery scenario without potential consistency violations. There are two primary problems with this assumption. First, disk errors are the most common problem we observe in real operation of persistent data systems and they often do not leave data intact. Secondly, even if this were not a problem, we do not want to require the use of fsync on every write for our consistency guarantees as this can reduce performance by two to three orders of magnitude. Our protocol for allowing a replica to rejoin the ISR ensures that before rejoining, it must fully re-sync again even if it lost unflushed data in its crash. + +### Unclean leader election: What if they all die? + +Note that Kafka's guarantee with respect to data loss is predicated on at least one replica remaining in sync. If all the nodes replicating a partition die, this guarantee no longer holds. + +However a practical system needs to do something reasonable when all the replicas die. If you are unlucky enough to have this occur, it is important to consider what will happen. There are two behaviors that could be implemented: + + 1. Wait for a replica in the ISR to come back to life and choose this replica as the leader (hopefully it still has all its data). + 2. Choose the first replica (not necessarily in the ISR) that comes back to life as the leader. + + +This is a simple tradeoff between availability and consistency. If we wait for replicas in the ISR, then we will remain unavailable as long as those replicas are down. If such replicas were destroyed or their data was lost, then we are permanently down. If, on the other hand, a non-in-sync replica comes back to life and we allow it to become leader, then its log becomes the source of truth even though it is not guaranteed to have every committed message. By default from version 0.11.0.0, Kafka chooses the first strategy and favor waiting for a consistent replica. This behavior can be changed using configuration property unclean.leader.election.enable, to support use cases where uptime is preferable to consistency. + +This dilemma is not specific to Kafka. It exists in any quorum-based scheme. For example in a majority voting scheme, if a majority of servers suffer a permanent failure, then you must either choose to lose 100% of your data or violate consistency by taking what remains on an existing server as your new source of truth. + +### Availability and Durability Guarantees + +When writing to Kafka, producers can choose whether they wait for the message to be acknowledged by 0,1 or all (-1) replicas. Note that "acknowledgement by all replicas" does not guarantee that the full set of assigned replicas have received the message. By default, when acks=all, acknowledgement happens as soon as all the current in-sync replicas have received the message. For example, if a topic is configured with only two replicas and one fails (i.e., only one in sync replica remains), then writes that specify acks=all will succeed. However, these writes could be lost if the remaining replica also fails. Although this ensures maximum availability of the partition, this behavior may be undesirable to some users who prefer durability over availability. Therefore, we provide two topic-level configurations that can be used to prefer message durability over availability: + + 1. Disable unclean leader election - if all replicas become unavailable, then the partition will remain unavailable until the most recent leader becomes available again. This effectively prefers unavailability over the risk of message loss. See the previous section on Unclean Leader Election for clarification. + 2. Specify a minimum ISR size - the partition will only accept writes if the size of the ISR is above a certain minimum, in order to prevent the loss of messages that were written to just a single replica, which subsequently becomes unavailable. This setting only takes effect if the producer uses acks=all and guarantees that the message will be acknowledged by at least this many in-sync replicas. This setting offers a trade-off between consistency and availability. A higher setting for minimum ISR size guarantees better consistency since the message is guaranteed to be written to more replicas which reduces the probability that it will be lost. However, it reduces availability since the partition will be unavailable for writes if the number of in-sync replicas drops below the minimum threshold. + + + +### Replica Management + +The above discussion on replicated logs really covers only a single log, i.e. one topic partition. However a Kafka cluster will manage hundreds or thousands of these partitions. We attempt to balance partitions within a cluster in a round-robin fashion to avoid clustering all partitions for high-volume topics on a small number of nodes. Likewise we try to balance leadership so that each node is the leader for a proportional share of its partitions. + +It is also important to optimize the leadership election process as that is the critical window of unavailability. A naive implementation of leader election would end up running an election per partition for all partitions a node hosted when that node failed. As discussed above in the section on replication, Kafka clusters have a special role known as the "controller" which is responsible for managing the registration of brokers. If the controller detects the failure of a broker, it is responsible for electing one of the remaining members of the ISR to serve as the new leader. The result is that we are able to batch together many of the required leadership change notifications which makes the election process far cheaper and faster for a large number of partitions. If the controller itself fails, then another controller will be elected. + +## Log Compaction + +Log compaction ensures that Kafka will always retain at least the last known value for each message key within the log of data for a single topic partition. It addresses use cases and scenarios such as restoring state after application crashes or system failure, or reloading caches after application restarts during operational maintenance. Let's dive into these use cases in more detail and then describe how compaction works. + +So far we have described only the simpler approach to data retention where old log data is discarded after a fixed period of time or when the log reaches some predetermined size. This works well for temporal event data such as logging where each record stands alone. However an important class of data streams are the log of changes to keyed, mutable data (for example, the changes to a database table). + +Let's discuss a concrete example of such a stream. Say we have a topic containing user email addresses; every time a user updates their email address we send a message to this topic using their user id as the primary key. Now say we send the following messages over some time period for a user with id 123, each message corresponding to a change in email address (messages for other ids are omitted): + + + 123 => bill@microsoft.com + . + . + . + 123 => bill@gatesfoundation.org + . + . + . + 123 => bill@gmail.com + +Log compaction gives us a more granular retention mechanism so that we are guaranteed to retain at least the last update for each primary key (e.g. `bill@gmail.com`). By doing this we guarantee that the log contains a full snapshot of the final value for every key not just keys that changed recently. This means downstream consumers can restore their own state off this topic without us having to retain a complete log of all changes. + +Let's start by looking at a few use cases where this is useful, then we'll see how it can be used. + + 1. _Database change subscription_. It is often necessary to have a data set in multiple data systems, and often one of these systems is a database of some kind (either a RDBMS or perhaps a new-fangled key-value store). For example you might have a database, a cache, a search cluster, and a Hadoop cluster. Each change to the database will need to be reflected in the cache, the search cluster, and eventually in Hadoop. In the case that one is only handling the real-time updates you only need recent log. But if you want to be able to reload the cache or restore a failed search node you may need a complete data set. + 2. _Event sourcing_. This is a style of application design which co-locates query processing with application design and uses a log of changes as the primary store for the application. + 3. _Journaling for high-availability_. A process that does local computation can be made fault-tolerant by logging out changes that it makes to its local state so another process can reload these changes and carry on if it should fail. A concrete example of this is handling counts, aggregations, and other "group by"-like processing in a stream query system. Samza, a real-time stream-processing framework, [uses this feature](https://samza.apache.org/learn/0.7.0/container/state-management.html) for exactly this purpose. +In each of these cases one needs primarily to handle the real-time feed of changes, but occasionally, when a machine crashes or data needs to be re-loaded or re-processed, one needs to do a full load. Log compaction allows feeding both of these use cases off the same backing topic. This style of usage of a log is described in more detail in [this blog post](https://engineering.linkedin.com/distributed-systems/log-what-every-software-engineer-should-know-about-real-time-datas-unifying). + +The general idea is quite simple. If we had infinite log retention, and we logged each change in the above cases, then we would have captured the state of the system at each time from when it first began. Using this complete log, we could restore to any point in time by replaying the first N records in the log. This hypothetical complete log is not very practical for systems that update a single record many times as the log will grow without bound even for a stable dataset. The simple log retention mechanism which throws away old updates will bound space but the log is no longer a way to restore the current state--now restoring from the beginning of the log no longer recreates the current state as old updates may not be captured at all. + +Log compaction is a mechanism to give finer-grained per-record retention, rather than the coarser-grained time-based retention. The idea is to selectively remove records where we have a more recent update with the same primary key. This way the log is guaranteed to have at least the last state for each key. + +This retention policy can be set per-topic, so a single cluster can have some topics where retention is enforced by size or time and other topics where retention is enforced by compaction. + +This functionality is inspired by one of LinkedIn's oldest and most successful pieces of infrastructure--a database changelog caching service called [Databus](https://github.com/linkedin/databus). Unlike most log-structured storage systems Kafka is built for subscription and organizes data for fast linear reads and writes. Unlike Databus, Kafka acts as a source-of-truth store so it is useful even in situations where the upstream data source would not otherwise be replayable. + +### Log Compaction Basics + +Here is a high-level picture that shows the logical structure of a Kafka log with the offset for each message. + +![](/37/images/log_cleaner_anatomy.png) + +The head of the log is identical to a traditional Kafka log. It has dense, sequential offsets and retains all messages. Log compaction adds an option for handling the tail of the log. The picture above shows a log with a compacted tail. Note that the messages in the tail of the log retain the original offset assigned when they were first written--that never changes. Note also that all offsets remain valid positions in the log, even if the message with that offset has been compacted away; in this case this position is indistinguishable from the next highest offset that does appear in the log. For example, in the picture above the offsets 36, 37, and 38 are all equivalent positions and a read beginning at any of these offsets would return a message set beginning with 38. + +Compaction also allows for deletes. A message with a key and a null payload will be treated as a delete from the log. Such a record is sometimes referred to as a _tombstone_. This delete marker will cause any prior message with that key to be removed (as would any new message with that key), but delete markers are special in that they will themselves be cleaned out of the log after a period of time to free up space. The point in time at which deletes are no longer retained is marked as the "delete retention point" in the above diagram. + +The compaction is done in the background by periodically recopying log segments. Cleaning does not block reads and can be throttled to use no more than a configurable amount of I/O throughput to avoid impacting producers and consumers. The actual process of compacting a log segment looks something like this: + +![](/37/images/log_compaction.png) + +### What guarantees does log compaction provide? + +Log compaction guarantees the following: + + 1. Any consumer that stays caught-up to within the head of the log will see every message that is written; these messages will have sequential offsets. The topic's `min.compaction.lag.ms` can be used to guarantee the minimum length of time must pass after a message is written before it could be compacted. I.e. it provides a lower bound on how long each message will remain in the (uncompacted) head. The topic's `max.compaction.lag.ms` can be used to guarantee the maximum delay between the time a message is written and the time the message becomes eligible for compaction. + 2. Ordering of messages is always maintained. Compaction will never re-order messages, just remove some. + 3. The offset for a message never changes. It is the permanent identifier for a position in the log. + 4. Any consumer progressing from the start of the log will see at least the final state of all records in the order they were written. Additionally, all delete markers for deleted records will be seen, provided the consumer reaches the head of the log in a time period less than the topic's `delete.retention.ms` setting (the default is 24 hours). In other words: since the removal of delete markers happens concurrently with reads, it is possible for a consumer to miss delete markers if it lags by more than `delete.retention.ms`. + + +### Log Compaction Details + +Log compaction is handled by the log cleaner, a pool of background threads that recopy log segment files, removing records whose key appears in the head of the log. Each compactor thread works as follows: + + 1. It chooses the log that has the highest ratio of log head to log tail + 2. It creates a succinct summary of the last offset for each key in the head of the log + 3. It recopies the log from beginning to end removing keys which have a later occurrence in the log. New, clean segments are swapped into the log immediately so the additional disk space required is just one additional log segment (not a fully copy of the log). + 4. The summary of the log head is essentially just a space-compact hash table. It uses exactly 24 bytes per entry. As a result with 8GB of cleaner buffer one cleaner iteration can clean around 366GB of log head (assuming 1k messages). + + +### Configuring The Log Cleaner + +The log cleaner is enabled by default. This will start the pool of cleaner threads. To enable log cleaning on a particular topic, add the log-specific property + + + log.cleanup.policy=compact + +The `log.cleanup.policy` property is a broker configuration setting defined in the broker's `server.properties` file; it affects all of the topics in the cluster that do not have a configuration override in place as documented [here](/documentation.html#brokerconfigs). The log cleaner can be configured to retain a minimum amount of the uncompacted "head" of the log. This is enabled by setting the compaction time lag. + + + log.cleaner.min.compaction.lag.ms + +This can be used to prevent messages newer than a minimum message age from being subject to compaction. If not set, all log segments are eligible for compaction except for the last segment, i.e. the one currently being written to. The active segment will not be compacted even if all of its messages are older than the minimum compaction time lag. The log cleaner can be configured to ensure a maximum delay after which the uncompacted "head" of the log becomes eligible for log compaction. + + + log.cleaner.max.compaction.lag.ms + +This can be used to prevent log with low produce rate from remaining ineligible for compaction for an unbounded duration. If not set, logs that do not exceed min.cleanable.dirty.ratio are not compacted. Note that this compaction deadline is not a hard guarantee since it is still subjected to the availability of log cleaner threads and the actual compaction time. You will want to monitor the uncleanable-partitions-count, max-clean-time-secs and max-compaction-delay-secs metrics. + +Further cleaner configurations are described [here](/documentation.html#brokerconfigs). + +## Quotas + +Kafka cluster has the ability to enforce quotas on requests to control the broker resources used by clients. Two types of client quotas can be enforced by Kafka brokers for each group of clients sharing a quota: + + 1. Network bandwidth quotas define byte-rate thresholds (since 0.9) + 2. Request rate quotas define CPU utilization thresholds as a percentage of network and I/O threads (since 0.11) + + + +### Why are quotas necessary? + +It is possible for producers and consumers to produce/consume very high volumes of data or generate requests at a very high rate and thus monopolize broker resources, cause network saturation and generally DOS other clients and the brokers themselves. Having quotas protects against these issues and is all the more important in large multi-tenant clusters where a small set of badly behaved clients can degrade user experience for the well behaved ones. In fact, when running Kafka as a service this even makes it possible to enforce API limits according to an agreed upon contract. + +### Client groups + +The identity of Kafka clients is the user principal which represents an authenticated user in a secure cluster. In a cluster that supports unauthenticated clients, user principal is a grouping of unauthenticated users chosen by the broker using a configurable `PrincipalBuilder`. Client-id is a logical grouping of clients with a meaningful name chosen by the client application. The tuple (user, client-id) defines a secure logical group of clients that share both user principal and client-id. + +Quotas can be applied to (user, client-id), user or client-id groups. For a given connection, the most specific quota matching the connection is applied. All connections of a quota group share the quota configured for the group. For example, if (user="test-user", client-id="test-client") has a produce quota of 10MB/sec, this is shared across all producer instances of user "test-user" with the client-id "test-client". + +### Quota Configuration + +Quota configuration may be defined for (user, client-id), user and client-id groups. It is possible to override the default quota at any of the quota levels that needs a higher (or even lower) quota. The mechanism is similar to the per-topic log config overrides. User and (user, client-id) quota overrides are written to ZooKeeper under _**/config/users**_ and client-id quota overrides are written under _**/config/clients**_. These overrides are read by all brokers and are effective immediately. This lets us change quotas without having to do a rolling restart of the entire cluster. See here for details. Default quotas for each group may also be updated dynamically using the same mechanism. + +The order of precedence for quota configuration is: + + 1. /config/users//clients/ + 2. /config/users//clients/ + 3. /config/users/ + 4. /config/users//clients/ + 5. /config/users//clients/ + 6. /config/users/ + 7. /config/clients/ + 8. /config/clients/ + + + +### Network Bandwidth Quotas + +Network bandwidth quotas are defined as the byte rate threshold for each group of clients sharing a quota. By default, each unique client group receives a fixed quota in bytes/sec as configured by the cluster. This quota is defined on a per-broker basis. Each group of clients can publish/fetch a maximum of X bytes/sec per broker before clients are throttled. + +### Request Rate Quotas + +Request rate quotas are defined as the percentage of time a client can utilize on request handler I/O threads and network threads of each broker within a quota window. A quota of `n%` represents `n%` of one thread, so the quota is out of a total capacity of `((num.io.threads + num.network.threads) * 100)%`. Each group of clients may use a total percentage of upto `n%` across all I/O and network threads in a quota window before being throttled. Since the number of threads allocated for I/O and network threads are typically based on the number of cores available on the broker host, request rate quotas represent the total percentage of CPU that may be used by each group of clients sharing the quota. + +### Enforcement + +By default, each unique client group receives a fixed quota as configured by the cluster. This quota is defined on a per-broker basis. Each client can utilize this quota per broker before it gets throttled. We decided that defining these quotas per broker is much better than having a fixed cluster wide bandwidth per client because that would require a mechanism to share client quota usage among all the brokers. This can be harder to get right than the quota implementation itself! + +How does a broker react when it detects a quota violation? In our solution, the broker first computes the amount of delay needed to bring the violating client under its quota and returns a response with the delay immediately. In case of a fetch request, the response will not contain any data. Then, the broker mutes the channel to the client, not to process requests from the client anymore, until the delay is over. Upon receiving a response with a non-zero delay duration, the Kafka client will also refrain from sending further requests to the broker during the delay. Therefore, requests from a throttled client are effectively blocked from both sides. Even with older client implementations that do not respect the delay response from the broker, the back pressure applied by the broker via muting its socket channel can still handle the throttling of badly behaving clients. Those clients who sent further requests to the throttled channel will receive responses only after the delay is over. + +Byte-rate and thread utilization are measured over multiple small windows (e.g. 30 windows of 1 second each) in order to detect and correct quota violations quickly. Typically, having large measurement windows (for e.g. 10 windows of 30 seconds each) leads to large bursts of traffic followed by long delays which is not great in terms of user experience. diff --git a/docs/design/protocol.md b/docs/design/protocol.md new file mode 100644 index 0000000000000..26b980c300686 --- /dev/null +++ b/docs/design/protocol.md @@ -0,0 +1,193 @@ +--- +title: Protocol +description: +weight: 2 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +# Kafka protocol guide + +This document covers the wire protocol implemented in Kafka. It is meant to give a readable guide to the protocol that covers the available requests, their binary format, and the proper way to make use of them to implement a client. This document assumes you understand the basic design and terminology described [here](https://kafka.apache.org/documentation.html#design) + + * Preliminaries + * Network + * Partitioning and bootstrapping + * Partitioning Strategies + * Batching + * Versioning and Compatibility + * Retrieving Supported API versions + * SASL Authentication Sequence + * The Protocol + * Protocol Primitive Types + * Notes on reading the request format grammars + * Common Request and Response Structure + * Record Batch + * Evolving the Protocol + * The Request Header + * Versioning + * Constants + * Error Codes + * Api Keys + * The Messages + * Some Common Philosophical Questions + + + +### Preliminaries + +#### Network + +Kafka uses a binary protocol over TCP. The protocol defines all APIs as request response message pairs. All messages are size delimited and are made up of the following primitive types. + +The client initiates a socket connection and then writes a sequence of request messages and reads back the corresponding response message. No handshake is required on connection or disconnection. TCP is happier if you maintain persistent connections used for many requests to amortize the cost of the TCP handshake, but beyond this penalty connecting is pretty cheap. + +The client will likely need to maintain a connection to multiple brokers, as data is partitioned and the clients will need to talk to the server that has their data. However it should not generally be necessary to maintain multiple connections to a single broker from a single client instance (i.e. connection pooling). + +The server guarantees that on a single TCP connection, requests will be processed in the order they are sent and responses will return in that order as well. The broker's request processing allows only a single in-flight request per connection in order to guarantee this ordering. Note that clients can (and ideally should) use non-blocking IO to implement request pipelining and achieve higher throughput. i.e., clients can send requests even while awaiting responses for preceding requests since the outstanding requests will be buffered in the underlying OS socket buffer. All requests are initiated by the client, and result in a corresponding response message from the server except where noted. + +The server has a configurable maximum limit on request size and any request that exceeds this limit will result in the socket being disconnected. + +#### Partitioning and bootstrapping + +Kafka is a partitioned system so not all servers have the complete data set. Instead recall that topics are split into a pre-defined number of partitions, P, and each partition is replicated with some replication factor, N. Topic partitions themselves are just ordered "commit logs" numbered 0, 1, ..., P-1. + +All systems of this nature have the question of how a particular piece of data is assigned to a particular partition. Kafka clients directly control this assignment, the brokers themselves enforce no particular semantics of which messages should be published to a particular partition. Rather, to publish messages the client directly addresses messages to a particular partition, and when fetching messages, fetches from a particular partition. If two clients want to use the same partitioning scheme they must use the same method to compute the mapping of key to partition. + +These requests to publish or fetch data must be sent to the broker that is currently acting as the leader for a given partition. This condition is enforced by the broker, so a request for a particular partition to the wrong broker will result in an the NotLeaderForPartition error code (described below). + +How can the client find out which topics exist, what partitions they have, and which brokers currently host those partitions so that it can direct its requests to the right hosts? This information is dynamic, so you can't just configure each client with some static mapping file. Instead all Kafka brokers can answer a metadata request that describes the current state of the cluster: what topics there are, which partitions those topics have, which broker is the leader for those partitions, and the host and port information for these brokers. + +In other words, the client needs to somehow find one broker and that broker will tell the client about all the other brokers that exist and what partitions they host. This first broker may itself go down so the best practice for a client implementation is to take a list of two or three URLs to bootstrap from. The user can then choose to use a load balancer or just statically configure two or three of their Kafka hosts in the clients. + +The client does not need to keep polling to see if the cluster has changed; it can fetch metadata once when it is instantiated cache that metadata until it receives an error indicating that the metadata is out of date. This error can come in two forms: (1) a socket error indicating the client cannot communicate with a particular broker, (2) an error code in the response to a request indicating that this broker no longer hosts the partition for which data was requested. + + 1. Cycle through a list of "bootstrap" Kafka URLs until we find one we can connect to. Fetch cluster metadata. + 2. Process fetch or produce requests, directing them to the appropriate broker based on the topic/partitions they send to or fetch from. + 3. If we get an appropriate error, refresh the metadata and try again. + + + +#### Partitioning Strategies + +As mentioned above the assignment of messages to partitions is something the producing client controls. That said, how should this functionality be exposed to the end-user? + +Partitioning really serves two purposes in Kafka: + + 1. It balances data and request load over brokers + 2. It serves as a way to divvy up processing among consumer processes while allowing local state and preserving order within the partition. We call this semantic partitioning. + + + +For a given use case you may care about only one of these or both. + +To accomplish simple load balancing a simple approach would be for the client to just round robin requests over all brokers. Another alternative, in an environment where there are many more producers than brokers, would be to have each client chose a single partition at random and publish to that. This later strategy will result in far fewer TCP connections. + +Semantic partitioning means using some key in the message to assign messages to partitions. For example if you were processing a click message stream you might want to partition the stream by the user id so that all data for a particular user would go to a single consumer. To accomplish this the client can take a key associated with the message and use some hash of this key to choose the partition to which to deliver the message. + +#### Batching + +Our APIs encourage batching small things together for efficiency. We have found this is a very significant performance win. Both our API to send messages and our API to fetch messages always work with a sequence of messages not a single message to encourage this. A clever client can make use of this and support an "asynchronous" mode in which it batches together messages sent individually and sends them in larger clumps. We go even further with this and allow the batching across multiple topics and partitions, so a produce request may contain data to append to many partitions and a fetch request may pull data from many partitions all at once. + +The client implementer can choose to ignore this and send everything one at a time if they like. + +#### Compatibility + +Kafka has a "bidirectional" client compatibility policy. In other words, new clients can talk to old servers, and old clients can talk to new servers. This allows users to upgrade either clients or servers without experiencing any downtime. + +Since the Kafka protocol has changed over time, clients and servers need to agree on the schema of the message that they are sending over the wire. This is done through API versioning. + +Before each request is sent, the client sends the API key and the API version. These two 16-bit numbers, when taken together, uniquely identify the schema of the message to follow. + +The intention is that clients will support a range of API versions. When communicating with a particular broker, a given client should use the highest API version supported by both and indicate this version in their requests. + +The server will reject requests with a version it does not support, and will always respond to the client with exactly the protocol format it expects based on the version it included in its request. The intended upgrade path is that new features would first be rolled out on the server (with the older clients not making use of them) and then as newer clients are deployed these new features would gradually be taken advantage of. + +Note that [KIP-482 tagged fields](https://cwiki.apache.org/confluence/display/KAFKA/KIP-482%3A+The+Kafka+Protocol+should+Support+Optional+Tagged+Fields) can be added to a request without incrementing the version number. This offers an additional way of evolving the message schema without breaking compatibility. Tagged fields do not take up any space when the field is not set. Therefore, if a field is rarely used, it is more efficient to make it a tagged field than to put it in the mandatory schema. However, tagged fields are ignored by recipients that don't know about them, which could pose a challenge if this is not the behavior that the sender wants. In such cases, a version bump may be more appropriate. + +#### Retrieving Supported API versions + +In order to work against multiple broker versions, clients need to know what versions of various APIs a broker supports. The broker exposes this information since 0.10.0.0 as described in [KIP-35](https://cwiki.apache.org/confluence/display/KAFKA/KIP-35+-+Retrieving+protocol+version). Clients should use the supported API versions information to choose the highest API version supported by both client and broker. If no such version exists, an error should be reported to the user. + +The following sequence may be used by a client to obtain supported API versions from a broker. + + 1. Client sends `ApiVersionsRequest` to a broker after connection has been established with the broker. If SSL is enabled, this happens after SSL connection has been established. + 2. On receiving `ApiVersionsRequest`, a broker returns its full list of supported ApiKeys and versions regardless of current authentication state (e.g., before SASL authentication on an SASL listener, do note that no Kafka protocol requests may take place on an SSL listener before the SSL handshake is finished). If this is considered to leak information about the broker version a workaround is to use SSL with client authentication which is performed at an earlier stage of the connection where the `ApiVersionRequest` is not available. Also, note that broker versions older than 0.10.0.0 do not support this API and will either ignore the request or close connection in response to the request. + 3. If multiple versions of an API are supported by broker and client, clients are recommended to use the latest version supported by the broker and itself. + 4. Deprecation of a protocol version is done by marking an API version as deprecated in the protocol documentation. + 5. Supported API versions obtained from a broker are only valid for the connection on which that information is obtained. In the event of disconnection, the client should obtain the information from the broker again, as the broker might have been upgraded/downgraded in the mean time. + + + +#### SASL Authentication Sequence + +The following sequence is used for SASL authentication: + + 1. Kafka `ApiVersionsRequest` may be sent by the client to obtain the version ranges of requests supported by the broker. This is optional. + 2. Kafka `SaslHandshakeRequest` containing the SASL mechanism for authentication is sent by the client. If the requested mechanism is not enabled in the server, the server responds with the list of supported mechanisms and closes the client connection. If the mechanism is enabled in the server, the server sends a successful response and continues with SASL authentication. + 3. The actual SASL authentication is now performed. If `SaslHandshakeRequest` version is v0, a series of SASL client and server tokens corresponding to the mechanism are sent as opaque packets without wrapping the messages with Kafka protocol headers. If `SaslHandshakeRequest` version is v1, the `SaslAuthenticate` request/response are used, where the actual SASL tokens are wrapped in the Kafka protocol. The error code in the final message from the broker will indicate if authentication succeeded or failed. + 4. If authentication succeeds, subsequent packets are handled as Kafka API requests. Otherwise, the client connection is closed. + + + +For interoperability with 0.9.0.x clients, the first packet received by the server is handled as a SASL/GSSAPI client token if it is not a valid Kafka request. SASL/GSSAPI authentication is performed starting with this packet, skipping the first two steps above. + +### The Protocol + +#### Protocol Primitive Types + +The protocol is built out of the following primitive types. + +{{< include-html file="/static/37/generated/protocol_types.html" >}} + +#### Notes on reading the request format grammars + +The [BNF](https://en.wikipedia.org/wiki/Backus%E2%80%93Naur_Form)s below give an exact context free grammar for the request and response binary format. The BNF is intentionally not compact in order to give human-readable name. As always in a BNF a sequence of productions indicates concatenation. When there are multiple possible productions these are separated with '|' and may be enclosed in parenthesis for grouping. The top-level definition is always given first and subsequent sub-parts are indented. + +#### Common Request and Response Structure + +All requests and responses originate from the following grammar which will be incrementally describe through the rest of this document: + + + RequestOrResponse => Size (RequestMessage | ResponseMessage) + Size => int32 + +Field| Description +---|--- +message_size| The message_size field gives the size of the subsequent request or response message in bytes. The client can read requests by first reading this 4 byte size as an integer N, and then reading and parsing the subsequent N bytes of the request. + +#### Record Batch + +A description of the record batch format can be found [here](/#recordbatch). + +### Constants + +#### Error Codes + +We use numeric codes to indicate what problem occurred on the server. These can be translated by the client into exceptions or whatever the appropriate error handling mechanism in the client language. Here is a table of the error codes currently in use: + +{{< include-html file="/static/37/generated/protocol_errors.html" >}} + +#### Api Keys + +The following are the numeric codes that the ApiKey in the request can take for each of the below request types. + +{{< include-html file="/static/37/generated/protocol_api_keys.html" >}} + +### The Messages + +This section gives details on each of the individual API Messages, their usage, their binary format, and the meaning of their fields. + +{{< include-html file="/static/37/generated/protocol_messages.html" >}} + +### Some Common Philosophical Questions + +Some people have asked why we don't use HTTP. There are a number of reasons, the best is that client implementors can make use of some of the more advanced TCP features--the ability to multiplex requests, the ability to simultaneously poll many connections, etc. We have also found HTTP libraries in many languages to be surprisingly shabby. + +Others have asked if maybe we shouldn't support many different protocols. Prior experience with this was that it makes it very hard to add and test new features if they have to be ported across many protocol implementations. Our feeling is that most users don't really see multiple protocols as a feature, they just want a good reliable client in the language of their choice. + +Another question is why we don't adopt XMPP, STOMP, AMQP or an existing protocol. The answer to this varies by protocol, but in general the problem is that the protocol does determine large parts of the implementation and we couldn't do what we are doing if we didn't have control over the protocol. Our belief is that it is possible to do better than existing messaging systems have in providing a truly distributed messaging system, and to do this we need to build something that works differently. + +A final question is why we don't use a system like Protocol Buffers or Thrift to define our request messages. These packages excel at helping you to managing lots and lots of serialized messages. However we have only a few messages. Support across languages is somewhat spotty (depending on the package). Finally the mapping between binary log format and wire protocol is something we manage somewhat carefully and this would not be possible with these systems. Finally we prefer the style of versioning APIs explicitly and checking this to inferring new values as nulls as it allows more nuanced control of compatibility. diff --git a/docs/docker.html b/docs/docker.html deleted file mode 100644 index bbedb10284103..0000000000000 --- a/docs/docker.html +++ /dev/null @@ -1,58 +0,0 @@ - - - - - - -
\ No newline at end of file diff --git a/docs/documentation.html b/docs/documentation.html deleted file mode 100644 index cd6373e7846f0..0000000000000 --- a/docs/documentation.html +++ /dev/null @@ -1,116 +0,0 @@ - - - - - - - - -
- -
-
<
-
-
- -
-
- - -

Documentation

-

Kafka 3.7 Documentation

- Prior releases: 0.7.x, - 0.8.0, - 0.8.1.X, - 0.8.2.X, - 0.9.0.X, - 0.10.0.X, - 0.10.1.X, - 0.10.2.X, - 0.11.0.X, - 1.0.X, - 1.1.X, - 2.0.X, - 2.1.X, - 2.2.X, - 2.3.X, - 2.4.X, - 2.5.X, - 2.6.X, - 2.7.X, - 2.8.X, - 3.0.X. - 3.1.X. - 3.2.X. - 3.3.X. - 3.4.X. - 3.5.X. - 3.6.X. - -

1. Getting Started

-

1.1 Introduction

- -

1.2 Use Cases

- -

1.3 Quick Start

- -

1.4 Ecosystem

- -

1.5 Upgrading From Previous Versions

- -

1.6 Docker

- - -

2. APIs

- - - -

3. Configuration

- - - -

4. Design

- - - -

5. Implementation

- - - -

6. Operations

- - - -

7. Security

- - -

8. Kafka Connect

- - -

9. Kafka Streams

-

- Kafka Streams is a client library for processing and analyzing data stored in Kafka. It builds upon important stream processing concepts such as properly distinguishing between event time and processing time, windowing support, exactly-once processing semantics and simple yet efficient management of application state. -

-

- Kafka Streams has a low barrier to entry: You can quickly write and run a small-scale proof-of-concept on a single machine; and you only need to run additional instances of your application on multiple machines to scale up to high-volume production workloads. Kafka Streams transparently handles the load balancing of multiple instances of the same application by leveraging Kafka's parallelism model. -

- -

Learn More about Kafka Streams read this Section.

- - - diff --git a/docs/documentation/index.html b/docs/documentation/index.html deleted file mode 100644 index 1d7507f30a9c4..0000000000000 --- a/docs/documentation/index.html +++ /dev/null @@ -1,18 +0,0 @@ - - - \ No newline at end of file diff --git a/docs/documentation/streams/architecture.html b/docs/documentation/streams/architecture.html deleted file mode 100644 index ad7b323d077fd..0000000000000 --- a/docs/documentation/streams/architecture.html +++ /dev/null @@ -1,19 +0,0 @@ - - - - diff --git a/docs/documentation/streams/core-concepts.html b/docs/documentation/streams/core-concepts.html deleted file mode 100644 index d699b795000b5..0000000000000 --- a/docs/documentation/streams/core-concepts.html +++ /dev/null @@ -1,19 +0,0 @@ - - - - diff --git a/docs/documentation/streams/developer-guide/app-reset-tool.html b/docs/documentation/streams/developer-guide/app-reset-tool.html deleted file mode 100644 index 64a43aa0c325d..0000000000000 --- a/docs/documentation/streams/developer-guide/app-reset-tool.html +++ /dev/null @@ -1,19 +0,0 @@ - - - - diff --git a/docs/documentation/streams/developer-guide/config-streams.html b/docs/documentation/streams/developer-guide/config-streams.html deleted file mode 100644 index 979f66d68a16d..0000000000000 --- a/docs/documentation/streams/developer-guide/config-streams.html +++ /dev/null @@ -1,19 +0,0 @@ - - - - diff --git a/docs/documentation/streams/developer-guide/datatypes.html b/docs/documentation/streams/developer-guide/datatypes.html deleted file mode 100644 index 98dd3a1c73a1c..0000000000000 --- a/docs/documentation/streams/developer-guide/datatypes.html +++ /dev/null @@ -1,19 +0,0 @@ - - - - diff --git a/docs/documentation/streams/developer-guide/dsl-api.html b/docs/documentation/streams/developer-guide/dsl-api.html deleted file mode 100644 index 1bbc06d5d9471..0000000000000 --- a/docs/documentation/streams/developer-guide/dsl-api.html +++ /dev/null @@ -1,19 +0,0 @@ - - - - diff --git a/docs/documentation/streams/developer-guide/dsl-topology-naming.html b/docs/documentation/streams/developer-guide/dsl-topology-naming.html deleted file mode 100644 index 9f42a04ac0ed9..0000000000000 --- a/docs/documentation/streams/developer-guide/dsl-topology-naming.html +++ /dev/null @@ -1,19 +0,0 @@ - - - - diff --git a/docs/documentation/streams/developer-guide/index.html b/docs/documentation/streams/developer-guide/index.html deleted file mode 100644 index 3a612477663fb..0000000000000 --- a/docs/documentation/streams/developer-guide/index.html +++ /dev/null @@ -1,19 +0,0 @@ - - - - diff --git a/docs/documentation/streams/developer-guide/interactive-queries.html b/docs/documentation/streams/developer-guide/interactive-queries.html deleted file mode 100644 index 05060121ebe2a..0000000000000 --- a/docs/documentation/streams/developer-guide/interactive-queries.html +++ /dev/null @@ -1,19 +0,0 @@ - - - - diff --git a/docs/documentation/streams/developer-guide/manage-topics.html b/docs/documentation/streams/developer-guide/manage-topics.html deleted file mode 100644 index f4225541e39fb..0000000000000 --- a/docs/documentation/streams/developer-guide/manage-topics.html +++ /dev/null @@ -1,19 +0,0 @@ - - - - diff --git a/docs/documentation/streams/developer-guide/memory-mgmt.html b/docs/documentation/streams/developer-guide/memory-mgmt.html deleted file mode 100644 index 024e1374cefca..0000000000000 --- a/docs/documentation/streams/developer-guide/memory-mgmt.html +++ /dev/null @@ -1,19 +0,0 @@ - - - - diff --git a/docs/documentation/streams/developer-guide/processor-api.html b/docs/documentation/streams/developer-guide/processor-api.html deleted file mode 100644 index 9e9ab91f01918..0000000000000 --- a/docs/documentation/streams/developer-guide/processor-api.html +++ /dev/null @@ -1,19 +0,0 @@ - - - - diff --git a/docs/documentation/streams/developer-guide/running-app.html b/docs/documentation/streams/developer-guide/running-app.html deleted file mode 100644 index 05d5f0b1f0608..0000000000000 --- a/docs/documentation/streams/developer-guide/running-app.html +++ /dev/null @@ -1,19 +0,0 @@ - - - - diff --git a/docs/documentation/streams/developer-guide/security.html b/docs/documentation/streams/developer-guide/security.html deleted file mode 100644 index 5d6e5f0d97e2e..0000000000000 --- a/docs/documentation/streams/developer-guide/security.html +++ /dev/null @@ -1,19 +0,0 @@ - - - - diff --git a/docs/documentation/streams/developer-guide/testing.html b/docs/documentation/streams/developer-guide/testing.html deleted file mode 100644 index 4753e66ad1c41..0000000000000 --- a/docs/documentation/streams/developer-guide/testing.html +++ /dev/null @@ -1,19 +0,0 @@ - - - - diff --git a/docs/documentation/streams/developer-guide/write-streams.html b/docs/documentation/streams/developer-guide/write-streams.html deleted file mode 100644 index 976c6fe1405e2..0000000000000 --- a/docs/documentation/streams/developer-guide/write-streams.html +++ /dev/null @@ -1,19 +0,0 @@ - - - - diff --git a/docs/documentation/streams/index.html b/docs/documentation/streams/index.html deleted file mode 100644 index 5ff3b3b6fbe39..0000000000000 --- a/docs/documentation/streams/index.html +++ /dev/null @@ -1,19 +0,0 @@ - - - - diff --git a/docs/documentation/streams/quickstart.html b/docs/documentation/streams/quickstart.html deleted file mode 100644 index efb0234e2407a..0000000000000 --- a/docs/documentation/streams/quickstart.html +++ /dev/null @@ -1,19 +0,0 @@ - - - - diff --git a/docs/documentation/streams/tutorial.html b/docs/documentation/streams/tutorial.html deleted file mode 100644 index e2cf4016e7032..0000000000000 --- a/docs/documentation/streams/tutorial.html +++ /dev/null @@ -1,19 +0,0 @@ - - - - diff --git a/docs/documentation/streams/upgrade-guide.html b/docs/documentation/streams/upgrade-guide.html deleted file mode 100644 index b1b32007e513b..0000000000000 --- a/docs/documentation/streams/upgrade-guide.html +++ /dev/null @@ -1,19 +0,0 @@ - - - - diff --git a/docs/ecosystem.html b/docs/ecosystem.html deleted file mode 100644 index 5fbcec59ba2ee..0000000000000 --- a/docs/ecosystem.html +++ /dev/null @@ -1,18 +0,0 @@ - - -There are a plethora of tools that integrate with Kafka outside the main distribution. The ecosystem page lists many of these, including stream processing systems, Hadoop integration, monitoring, and deployment tools. diff --git a/docs/getting-started/_index.md b/docs/getting-started/_index.md new file mode 100644 index 0000000000000..c5eccc36a563d --- /dev/null +++ b/docs/getting-started/_index.md @@ -0,0 +1,10 @@ +--- +title: Getting Started +description: This section provides an overview of what Kafka is, why it is useful, and how to get started using it. +weight: 1 +tags: ['kafka', 'docs', 'getting-started'] +aliases: +keywords: +type: docs +--- + diff --git a/docs/getting-started/docker.md b/docs/getting-started/docker.md new file mode 100644 index 0000000000000..8240fb2aabfa3 --- /dev/null +++ b/docs/getting-started/docker.md @@ -0,0 +1,36 @@ +--- +title: Docker +description: +weight: 8 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +## Introduction + +[Docker](https://www.docker.com/) is a popular container runtime. Docker images for Apache Kafka can be found on [Docker Hub](https://hub.docker.com/r/apache/kafka) and are available from version 3.7.0. + +## Getting the kafka docker image + +Docker image can be pulled from Docker Hub using the following command:- + + + $ docker pull apache/kafka:3.7.2 + +If you want to fetch the latest version of the docker image use following command:- + + + $ docker pull apache/kafka:latest + +## Start kafka with default configs + +Run docker image on default port 9092:- + + + $ docker run -p 9092:9092 apache/kafka:3.7.2 + +## Usage guide + +Detailed instructions for using the docker image are mentioned [here](https://github.com/apache/kafka/blob/trunk/docker/examples/README.md). diff --git a/docs/getting-started/ecosystem.md b/docs/getting-started/ecosystem.md new file mode 100644 index 0000000000000..7dd256aacac0e --- /dev/null +++ b/docs/getting-started/ecosystem.md @@ -0,0 +1,11 @@ +--- +title: Ecosystem +description: +weight: 4 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +There are a plethora of tools that integrate with Kafka outside the main distribution. The [ ecosystem page](https://cwiki.apache.org/confluence/display/KAFKA/Ecosystem) lists many of these, including stream processing systems, Hadoop integration, monitoring, and deployment tools. diff --git a/docs/getting-started/introduction.md b/docs/getting-started/introduction.md new file mode 100644 index 0000000000000..5c9383746ddae --- /dev/null +++ b/docs/getting-started/introduction.md @@ -0,0 +1,92 @@ +--- +title: Introduction +description: +weight: 1 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +## What is event streaming? + +Event streaming is the digital equivalent of the human body's central nervous system. It is the technological foundation for the 'always-on' world where businesses are increasingly software-defined and automated, and where the user of software is more software. + +Technically speaking, event streaming is the practice of capturing data in real-time from event sources like databases, sensors, mobile devices, cloud services, and software applications in the form of streams of events; storing these event streams durably for later retrieval; manipulating, processing, and reacting to the event streams in real-time as well as retrospectively; and routing the event streams to different destination technologies as needed. Event streaming thus ensures a continuous flow and interpretation of data so that the right information is at the right place, at the right time. + +## What can I use event streaming for? + +Event streaming is applied to a [wide variety of use cases](/powered-by) across a plethora of industries and organizations. Its many examples include: + + * To process payments and financial transactions in real-time, such as in stock exchanges, banks, and insurances. + * To track and monitor cars, trucks, fleets, and shipments in real-time, such as in logistics and the automotive industry. + * To continuously capture and analyze sensor data from IoT devices or other equipment, such as in factories and wind parks. + * To collect and immediately react to customer interactions and orders, such as in retail, the hotel and travel industry, and mobile applications. + * To monitor patients in hospital care and predict changes in condition to ensure timely treatment in emergencies. + * To connect, store, and make available data produced by different divisions of a company. + * To serve as the foundation for data platforms, event-driven architectures, and microservices. + + + +## Apache Kafka® is an event streaming platform. What does that mean? + +Kafka combines three key capabilities so you can implement [your use cases](/powered-by) for event streaming end-to-end with a single battle-tested solution: + + 1. To **publish** (write) and **subscribe to** (read) streams of events, including continuous import/export of your data from other systems. + 2. To **store** streams of events durably and reliably for as long as you want. + 3. To **process** streams of events as they occur or retrospectively. + + + +And all this functionality is provided in a distributed, highly scalable, elastic, fault-tolerant, and secure manner. Kafka can be deployed on bare-metal hardware, virtual machines, and containers, and on-premises as well as in the cloud. You can choose between self-managing your Kafka environments and using fully managed services offered by a variety of vendors. + +## How does Kafka work in a nutshell? + +Kafka is a distributed system consisting of **servers** and **clients** that communicate via a high-performance [TCP network protocol](/protocol.html). It can be deployed on bare-metal hardware, virtual machines, and containers in on-premise as well as cloud environments. + +**Servers** : Kafka is run as a cluster of one or more servers that can span multiple datacenters or cloud regions. Some of these servers form the storage layer, called the brokers. Other servers run [Kafka Connect](/#connect) to continuously import and export data as event streams to integrate Kafka with your existing systems such as relational databases as well as other Kafka clusters. To let you implement mission-critical use cases, a Kafka cluster is highly scalable and fault-tolerant: if any of its servers fails, the other servers will take over their work to ensure continuous operations without any data loss. + +**Clients** : They allow you to write distributed applications and microservices that read, write, and process streams of events in parallel, at scale, and in a fault-tolerant manner even in the case of network problems or machine failures. Kafka ships with some such clients included, which are augmented by [dozens of clients](https://cwiki.apache.org/confluence/display/KAFKA/Clients) provided by the Kafka community: clients are available for Java and Scala including the higher-level [Kafka Streams](/streams/) library, for Go, Python, C/C++, and many other programming languages as well as REST APIs. + +## Main Concepts and Terminology + +An **event** records the fact that "something happened" in the world or in your business. It is also called record or message in the documentation. When you read or write data to Kafka, you do this in the form of events. Conceptually, an event has a key, value, timestamp, and optional metadata headers. Here's an example event: + + * Event key: "Alice" + * Event value: "Made a payment of $200 to Bob" + * Event timestamp: "Jun. 25, 2020 at 2:06 p.m." + + + +**Producers** are those client applications that publish (write) events to Kafka, and **consumers** are those that subscribe to (read and process) these events. In Kafka, producers and consumers are fully decoupled and agnostic of each other, which is a key design element to achieve the high scalability that Kafka is known for. For example, producers never need to wait for consumers. Kafka provides various [guarantees](/#semantics) such as the ability to process events exactly-once. + +Events are organized and durably stored in **topics**. Very simplified, a topic is similar to a folder in a filesystem, and the events are the files in that folder. An example topic name could be "payments". Topics in Kafka are always multi-producer and multi-subscriber: a topic can have zero, one, or many producers that write events to it, as well as zero, one, or many consumers that subscribe to these events. Events in a topic can be read as often as needed—unlike traditional messaging systems, events are not deleted after consumption. Instead, you define for how long Kafka should retain your events through a per-topic configuration setting, after which old events will be discarded. Kafka's performance is effectively constant with respect to data size, so storing data for a long time is perfectly fine. + +Topics are **partitioned** , meaning a topic is spread over a number of "buckets" located on different Kafka brokers. This distributed placement of your data is very important for scalability because it allows client applications to both read and write the data from/to many brokers at the same time. When a new event is published to a topic, it is actually appended to one of the topic's partitions. Events with the same event key (e.g., a customer or vehicle ID) are written to the same partition, and Kafka [guarantees](/#semantics) that any consumer of a given topic-partition will always read that partition's events in exactly the same order as they were written. + +![](/images/streams-and-tables-p1_p4.png) Figure: This example topic has four partitions P1–P4. Two different producer clients are publishing, independently from each other, new events to the topic by writing events over the network to the topic's partitions. Events with the same key (denoted by their color in the figure) are written to the same partition. Note that both producers can write to the same partition if appropriate. + +To make your data fault-tolerant and highly-available, every topic can be **replicated** , even across geo-regions or datacenters, so that there are always multiple brokers that have a copy of the data just in case things go wrong, you want to do maintenance on the brokers, and so on. A common production setting is a replication factor of 3, i.e., there will always be three copies of your data. This replication is performed at the level of topic-partitions. + +This primer should be sufficient for an introduction. The [Design](/#design) section of the documentation explains Kafka's various concepts in full detail, if you are interested. + +## Kafka APIs + +In addition to command line tooling for management and administration tasks, Kafka has five core APIs for Java and Scala: + + * The [Admin API](/documentation.html#adminapi) to manage and inspect topics, brokers, and other Kafka objects. + * The [Producer API](/documentation.html#producerapi) to publish (write) a stream of events to one or more Kafka topics. + * The [Consumer API](/documentation.html#consumerapi) to subscribe to (read) one or more topics and to process the stream of events produced to them. + * The [Kafka Streams API](/streams) to implement stream processing applications and microservices. It provides higher-level functions to process event streams, including transformations, stateful operations like aggregations and joins, windowing, processing based on event-time, and more. Input is read from one or more topics in order to generate output to one or more topics, effectively transforming the input streams to output streams. + * The [Kafka Connect API](/documentation.html#connect) to build and run reusable data import/export connectors that consume (read) or produce (write) streams of events from and to external systems and applications so they can integrate with Kafka. For example, a connector to a relational database like PostgreSQL might capture every change to a set of tables. However, in practice, you typically don't need to implement your own connectors because the Kafka community already provides hundreds of ready-to-use connectors. + + + +## Where to go from here + + * To get hands-on experience with Kafka, follow the [Quickstart](/quickstart). + * To understand Kafka in more detail, read the [Documentation](/). You also have your choice of [Kafka books and academic papers](/books-and-papers). + * Browse through the [Use Cases](/powered-by) to learn how other users in our world-wide community are getting value out of Kafka. + * Join a [local Kafka meetup group](/events) and [watch talks from Kafka Summit](https://kafka-summit.org/past-events/), the main conference of the Kafka community. + + diff --git a/docs/getting-started/quickstart.md b/docs/getting-started/quickstart.md new file mode 100644 index 0000000000000..0e92bab12a3be --- /dev/null +++ b/docs/getting-started/quickstart.md @@ -0,0 +1,219 @@ +--- +title: Quick Start +description: +weight: 3 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +## Step 1: Get Kafka + +[Download](https://www.apache.org/dyn/closer.cgi?path=/kafka/3.7.2/kafka_2.13-3.7.2.tgz) the latest Kafka release and extract it: + + + $ tar -xzf kafka_2.13-3.7.2.tgz + $ cd kafka_2.13-3.7.2 + +## Step 2: Start the Kafka environment + +NOTE: Your local environment must have Java 8+ installed. + +Apache Kafka can be started using ZooKeeper or KRaft. To get started with either configuration follow one of the sections below but not both. + +### Kafka with ZooKeeper + +Run the following commands in order to start all services in the correct order: + + + # Start the ZooKeeper service + $ bin/zookeeper-server-start.sh config/zookeeper.properties + +Open another terminal session and run: + + + # Start the Kafka broker service + $ bin/kafka-server-start.sh config/server.properties + +Once all services have successfully launched, you will have a basic Kafka environment running and ready to use. + +### Kafka with KRaft + +Kafka can be run using KRaft mode using local scripts and downloaded files or the docker image. Follow one of the sections below but not both to start the kafka server. + +### Using downloaded files + +Generate a Cluster UUID + + + $ KAFKA_CLUSTER_ID="$(bin/kafka-storage.sh random-uuid)" + +Format Log Directories + + + $ bin/kafka-storage.sh format -t $KAFKA_CLUSTER_ID -c config/kraft/server.properties + +Start the Kafka Server + + + $ bin/kafka-server-start.sh config/kraft/server.properties + +### Using docker image + +Get the docker image + + + $ docker pull apache/kafka:3.7.2 + +Start the kafka docker container + + + $ docker run -p 9092:9092 apache/kafka:3.7.2 + +Once the Kafka server has successfully launched, you will have a basic Kafka environment running and ready to use. + +## Step 3: Create a topic to store your events + +Kafka is a distributed _event streaming platform_ that lets you read, write, store, and process [_events_](/#messages) (also called _records_ or _messages_ in the documentation) across many machines. + +Example events are payment transactions, geolocation updates from mobile phones, shipping orders, sensor measurements from IoT devices or medical equipment, and much more. These events are organized and stored in [_topics_](/#intro_concepts_and_terms). Very simplified, a topic is similar to a folder in a filesystem, and the events are the files in that folder. + +So before you can write your first events, you must create a topic. Open another terminal session and run: + + + $ bin/kafka-topics.sh --create --topic quickstart-events --bootstrap-server localhost:9092 + +All of Kafka's command line tools have additional options: run the `kafka-topics.sh` command without any arguments to display usage information. For example, it can also show you [details such as the partition count](/#intro_concepts_and_terms) of the new topic: + + + $ bin/kafka-topics.sh --describe --topic quickstart-events --bootstrap-server localhost:9092 + Topic: quickstart-events TopicId: NPmZHyhbR9y00wMglMH2sg PartitionCount: 1 ReplicationFactor: 1 Configs: + Topic: quickstart-events Partition: 0 Leader: 0 Replicas: 0 Isr: 0 + +## Step 4: Write some events into the topic + +A Kafka client communicates with the Kafka brokers via the network for writing (or reading) events. Once received, the brokers will store the events in a durable and fault-tolerant manner for as long as you need—even forever. + +Run the console producer client to write a few events into your topic. By default, each line you enter will result in a separate event being written to the topic. + + + $ bin/kafka-console-producer.sh --topic quickstart-events --bootstrap-server localhost:9092 + This is my first event + This is my second event + +You can stop the producer client with `Ctrl-C` at any time. + +## Step 5: Read the events + +Open another terminal session and run the console consumer client to read the events you just created: + + + $ bin/kafka-console-consumer.sh --topic quickstart-events --from-beginning --bootstrap-server localhost:9092 + This is my first event + This is my second event + +You can stop the consumer client with `Ctrl-C` at any time. + +Feel free to experiment: for example, switch back to your producer terminal (previous step) to write additional events, and see how the events immediately show up in your consumer terminal. + +Because events are durably stored in Kafka, they can be read as many times and by as many consumers as you want. You can easily verify this by opening yet another terminal session and re-running the previous command again. + +## Step 6: Import/export your data as streams of events with Kafka Connect + +You probably have lots of data in existing systems like relational databases or traditional messaging systems, along with many applications that already use these systems. [Kafka Connect](/#connect) allows you to continuously ingest data from external systems into Kafka, and vice versa. It is an extensible tool that runs _connectors_ , which implement the custom logic for interacting with an external system. It is thus very easy to integrate existing systems with Kafka. To make this process even easier, there are hundreds of such connectors readily available. + +In this quickstart we'll see how to run Kafka Connect with simple connectors that import data from a file to a Kafka topic and export data from a Kafka topic to a file. + +First, make sure to add `connect-file-3.7.2.jar` to the `plugin.path` property in the Connect worker's configuration. For the purpose of this quickstart we'll use a relative path and consider the connectors' package as an uber jar, which works when the quickstart commands are run from the installation directory. However, it's worth noting that for production deployments using absolute paths is always preferable. See [plugin.path](/#connectconfigs_plugin.path) for a detailed description of how to set this config. + +Edit the `config/connect-standalone.properties` file, add or change the `plugin.path` configuration property match the following, and save the file: + + + > echo "plugin.path=libs/connect-file-3.7.2.jar" + +Then, start by creating some seed data to test with: + + + > echo -e "foo + bar" > test.txt + +Or on Windows: + + + > echo foo> test.txt + > echo bar>> test.txt + +Next, we'll start two connectors running in _standalone_ mode, which means they run in a single, local, dedicated process. We provide three configuration files as parameters. The first is always the configuration for the Kafka Connect process, containing common configuration such as the Kafka brokers to connect to and the serialization format for data. The remaining configuration files each specify a connector to create. These files include a unique connector name, the connector class to instantiate, and any other configuration required by the connector. + + + > bin/connect-standalone.sh config/connect-standalone.properties config/connect-file-source.properties config/connect-file-sink.properties + +These sample configuration files, included with Kafka, use the default local cluster configuration you started earlier and create two connectors: the first is a source connector that reads lines from an input file and produces each to a Kafka topic and the second is a sink connector that reads messages from a Kafka topic and produces each as a line in an output file. + +During startup you'll see a number of log messages, including some indicating that the connectors are being instantiated. Once the Kafka Connect process has started, the source connector should start reading lines from `test.txt` and producing them to the topic `connect-test`, and the sink connector should start reading messages from the topic `connect-test` and write them to the file `test.sink.txt`. We can verify the data has been delivered through the entire pipeline by examining the contents of the output file: + + + > more test.sink.txt + foo + bar + +Note that the data is being stored in the Kafka topic `connect-test`, so we can also run a console consumer to see the data in the topic (or use custom consumer code to process it): + + + > bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 --topic connect-test --from-beginning + {"schema":{"type":"string","optional":false},"payload":"foo"} + {"schema":{"type":"string","optional":false},"payload":"bar"} + ... + +The connectors continue to process data, so we can add data to the file and see it move through the pipeline: + + + > echo Another line>> test.txt + +You should see the line appear in the console consumer output and in the sink file. + +## Step 7: Process your events with Kafka Streams + +Once your data is stored in Kafka as events, you can process the data with the [Kafka Streams](/streams) client library for Java/Scala. It allows you to implement mission-critical real-time applications and microservices, where the input and/or output data is stored in Kafka topics. Kafka Streams combines the simplicity of writing and deploying standard Java and Scala applications on the client side with the benefits of Kafka's server-side cluster technology to make these applications highly scalable, elastic, fault-tolerant, and distributed. The library supports exactly-once processing, stateful operations and aggregations, windowing, joins, processing based on event-time, and much more. + +To give you a first taste, here's how one would implement the popular `WordCount` algorithm: + + + KStream textLines = builder.stream("quickstart-events"); + + KTable wordCounts = textLines + .flatMapValues(line -> Arrays.asList(line.toLowerCase().split(" "))) + .groupBy((keyIgnored, word) -> word) + .count(); + + wordCounts.toStream().to("output-topic", Produced.with(Serdes.String(), Serdes.Long())); + +The [Kafka Streams demo](/streams/quickstart) and the [app development tutorial](/37/streams/tutorial) demonstrate how to code and run such a streaming application from start to finish. + +## Step 8: Terminate the Kafka environment + +Now that you reached the end of the quickstart, feel free to tear down the Kafka environment—or continue playing around. + + 1. Stop the producer and consumer clients with `Ctrl-C`, if you haven't done so already. + 2. Stop the Kafka broker with `Ctrl-C`. + 3. Lastly, if the Kafka with ZooKeeper section was followed, stop the ZooKeeper server with `Ctrl-C`. + + + +If you also want to delete any data of your local Kafka environment including any events you have created along the way, run the command: + + + $ rm -rf /tmp/kafka-logs /tmp/zookeeper /tmp/kraft-combined-logs + +## Congratulations! + +You have successfully finished the Apache Kafka quickstart. + +To learn more, we suggest the following next steps: + + * Read through the brief [Introduction](/intro) to learn how Kafka works at a high level, its main concepts, and how it compares to other technologies. To understand Kafka in more detail, head over to the [Documentation](/). + * Browse through the [Use Cases](/powered-by) to learn how other users in our world-wide community are getting value out of Kafka. + * Join a [local Kafka meetup group](/events) and [watch talks from Kafka Summit](https://kafka-summit.org/past-events/), the main conference of the Kafka community. + + diff --git a/docs/getting-started/upgrade.md b/docs/getting-started/upgrade.md new file mode 100644 index 0000000000000..6dcc98b482702 --- /dev/null +++ b/docs/getting-started/upgrade.md @@ -0,0 +1,1325 @@ +--- +title: Upgrading +description: +weight: 5 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +## Upgrading to 3.7.2 from any version 0.8.x through 3.6.x + +### Notable changes in 3.7.2 + + * In case you run your Kafka clusters with no execution permission for the `/tmp` partition, Kafka will not work properly. It might either refuse to start or fail when producing and consuming messages. This is due to the compression libraries `zstd-jni` and `snappy`. To remediate this problem you need to pass the following JVM flags to Kafka `ZstdTempFolder` and `org.xerial.snappy.tempdir` pointing to a directory with execution permissions. For example, this could be done via the `KAFKA_OPTS` environment variable like follows: `export KAFKA_OPTS="-DZstdTempFolder=/opt/kafka/tmp -Dorg.xerial.snappy.tempdir=/opt/kafka/tmp"`. + + + +## Upgrading to 3.7.0 from any version 0.8.x through 3.6.x + +### Upgrading ZooKeeper-based clusters + +**If you are upgrading from a version prior to 2.1.x, please see the note in step 5 below about the change to the schema used to store consumer offsets. Once you have changed the inter.broker.protocol.version to the latest version, it will not be possible to downgrade to a version prior to 2.1.** + +**For a rolling upgrade:** + + 1. Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. `3.6`, `3.5`, etc.) + * log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact following the upgrade for the details on what this configuration does.) +If you are upgrading from version 0.11.0.x or above, and you have not overridden the message format, then you only need to override the inter-broker protocol version. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. `3.6`, `3.5`, etc.) + 2. Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. Once you have done so, the brokers will be running the latest version and you can verify that the cluster's behavior and performance meets expectations. It is still possible to downgrade at this point if there are any problems. + 3. Once the cluster's behavior and performance has been verified, bump the protocol version by editing `inter.broker.protocol.version` and setting it to `3.7`. + 4. Restart the brokers one by one for the new protocol version to take effect. Once the brokers begin using the latest protocol version, it will no longer be possible to downgrade the cluster to an older version. + 5. If you have overridden the message format version as instructed above, then you need to do one more rolling restart to upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later, change log.message.format.version to 3.7 on each broker and restart them one by one. Note that the older Scala clients, which are no longer maintained, do not support the message format introduced in 0.11, so to avoid conversion costs (or to take advantage of exactly once semantics), the newer Java clients must be used. + + + +### Upgrading KRaft-based clusters + +**If you are upgrading from a version prior to 3.3.0, please see the note in step 3 below. Once you have changed the metadata.version to the latest version, it will not be possible to downgrade to a version prior to 3.3-IV0.** + +**For a rolling upgrade:** + + 1. Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. Once you have done so, the brokers will be running the latest version and you can verify that the cluster's behavior and performance meets expectations. + 2. Once the cluster's behavior and performance has been verified, bump the metadata.version by running ` ./bin/kafka-features.sh upgrade --metadata 3.7 ` + 3. Note that cluster metadata downgrade is not supported in this version since it has metadata changes. Every [MetadataVersion](https://github.com/apache/kafka/blob/trunk/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java) after 3.2.x has a boolean parameter that indicates if there are metadata changes (i.e. `IBP_3_3_IV3(7, "3.3", "IV3", true)` means this version has metadata changes). Given your current and target versions, a downgrade is only possible if there are no metadata changes in the versions between. + + + +### Notable changes in 3.7.2 + + * The consumer gets an important bug-fix for KIP-714. See [KAFKA-17731](https://issues.apache.org/jira/browse/KAFKA-17731) for more details. + * Kafka Streams, using EOSv2, might commit incorrect offsets potentially leading to data loss in repartition topics. See [KAFKA-17635](https://issues.apache.org/jira/browse/KAFKA-17635) for more details. + * Kafka Streams might pause(), but never resume() partitions. See [KAFKA-17299](https://issues.apache.org/jira/browse/KAFKA-17299) for more details. + + + +### Notable changes in 3.7.1 + + * MirrorMaker 2 can now emit checkpoints for offsets mirrored before the start of the Checkpoint task for improved offset translation. This requires MirrorMaker 2 to have READ authorization to the Checkpoint topic. If READ is not authorized, checkpointing is limited to offsets mirrorred after the start of the task. See [KAFKA-15905](https://issues.apache.org/jira/browse/KAFKA-15905) for more details. + * JBOD support in KRaft was introduced from Metadata Version (MV) 3.7-IV2. Configuring Brokers with multiple log directories can lead to indefinite unavailability. Brokers will now detect this situation and log an error. See [KAFKA-16606](https://issues.apache.org/jira/browse/KAFKA-16606) for more details. + + + +### Notable changes in 3.7.0 + + * Java 11 support for the broker and tools has been deprecated and will be removed in Apache Kafka 4.0. This complements the previous deprecation of Java 8 for all components. Please refer to [KIP-1013](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=284789510) for more details. + * Client APIs released prior to Apache Kafka 2.1 are now marked deprecated in 3.7 and will be removed in Apache Kafka 4.0. See [KIP-896](https://cwiki.apache.org/confluence/display/KAFKA/KIP-896%3A+Remove+old+client+protocol+API+versions+in+Kafka+4.0) for details and RPC versions that are now deprecated. + * Early access of the new simplified Consumer Rebalance Protocol is available, and it is not recommended for use in production environments. You are encouraged to test it and provide feedback! For more information about the early access feature, please check [KIP-848](https://cwiki.apache.org/confluence/display/KAFKA/KIP-848%3A+The+Next+Generation+of+the+Consumer+Rebalance+Protocol) and the [Early Access Release Notes](https://cwiki.apache.org/confluence/display/KAFKA/The+Next+Generation+of+the+Consumer+Rebalance+Protocol+%28KIP-848%29+-+Early+Access+Release+Notes). + * More metrics related to Tiered Storage have been introduced. They should improve the operational experience of running Tiered Storage in production. For more detailed information, please refer to [KIP-963](https://cwiki.apache.org/confluence/display/KAFKA/KIP-963%3A+Additional+metrics+in+Tiered+Storage). + * Kafka Streams ships multiple KIPs for IQv2 support. See the [Kafka Streams upgrade section](/37/streams/upgrade-guide#streams_api_changes_370) for more details. + * All the notable changes are present in the [blog post announcing the 3.7.0 release.](https://kafka.apache.org/blog#apache_kafka_370_release_announcement) + + + +## Upgrading to 3.6.2 from any version 0.8.x through 3.5.x + +### Upgrading ZooKeeper-based clusters + +**If you are upgrading from a version prior to 2.1.x, please see the note in step 5 below about the change to the schema used to store consumer offsets. Once you have changed the inter.broker.protocol.version to the latest version, it will not be possible to downgrade to a version prior to 2.1.** + +**For a rolling upgrade:** + + 1. Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. `3.5`, `3.4`, etc.) + * log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact following the upgrade for the details on what this configuration does.) +If you are upgrading from version 0.11.0.x or above, and you have not overridden the message format, then you only need to override the inter-broker protocol version. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. `3.5`, `3.4`, etc.) + 2. Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. Once you have done so, the brokers will be running the latest version and you can verify that the cluster's behavior and performance meets expectations. It is still possible to downgrade at this point if there are any problems. + 3. Once the cluster's behavior and performance has been verified, bump the protocol version by editing `inter.broker.protocol.version` and setting it to `3.6`. + 4. Restart the brokers one by one for the new protocol version to take effect. Once the brokers begin using the latest protocol version, it will no longer be possible to downgrade the cluster to an older version. + 5. If you have overridden the message format version as instructed above, then you need to do one more rolling restart to upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later, change log.message.format.version to 3.6 on each broker and restart them one by one. Note that the older Scala clients, which are no longer maintained, do not support the message format introduced in 0.11, so to avoid conversion costs (or to take advantage of exactly once semantics), the newer Java clients must be used. + + + +### Upgrading KRaft-based clusters + +**If you are upgrading from a version prior to 3.3.0, please see the note in step 3 below. Once you have changed the metadata.version to the latest version, it will not be possible to downgrade to a version prior to 3.3-IV0.** + +**For a rolling upgrade:** + + 1. Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. Once you have done so, the brokers will be running the latest version and you can verify that the cluster's behavior and performance meets expectations. + 2. Once the cluster's behavior and performance has been verified, bump the metadata.version by running ` ./bin/kafka-features.sh upgrade --metadata 3.6 ` + 3. Note that cluster metadata downgrade is not supported in this version since it has metadata changes. Every [MetadataVersion](https://github.com/apache/kafka/blob/trunk/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java) after 3.2.x has a boolean parameter that indicates if there are metadata changes (i.e. `IBP_3_3_IV3(7, "3.3", "IV3", true)` means this version has metadata changes). Given your current and target versions, a downgrade is only possible if there are no metadata changes in the versions between. + + + +### Notable changes in 3.6.0 + + * Apache Kafka now supports having both an IPv4 and an IPv6 listener on the same port. This change only applies to non advertised listeners (advertised listeners already have this feature) + * The Apache Zookeeper dependency has been upgraded to 3.8.1 due to 3.6 reaching end-of-life. To bring both your Kafka and Zookeeper clusters to the latest versions: + * **>=2.4** Kafka clusters can be updated directly. Zookeeper clusters which are running binaries bundled with Kafka versions 2.4 or above can be updated directly. + * **< 2.4** Kafka clusters first need to be updated to a version greater than 2.4 and smaller than 3.6. Zookeeper clusters which are running binaries bundled with Kafka versions below 2.4 need to be updated to any binaries bundled with Kafka versions greater than 2.4 and smaller than 3.6. You can then follow the first bullet-point. +For more detailed information please refer to the Compatibility, Deprecation, and Migration Plan section in [KIP-902](https://cwiki.apache.org/confluence/display/KAFKA/KIP-902%3A+Upgrade+Zookeeper+to+3.8.1). + * The configuration `log.message.timestamp.difference.max.ms` is deprecated. Two new configurations, `log.message.timestamp.before.max.ms` and `log.message.timestamp.after.max.ms`, have been added. For more detailed information, please refer to [KIP-937](https://cwiki.apache.org/confluence/display/KAFKA/KIP-937%3A+Improve+Message+Timestamp+Validation). + * Kafka Streams has introduced a new task assignor, `RackAwareTaskAssignor`, for computing task assignments which can minimize cross rack traffic under certain conditions. It works with existing `StickyTaskAssignor` and `HighAvailabilityTaskAssignor`. See [KIP-925](https://cwiki.apache.org/confluence/display/KAFKA/KIP-925%3A+Rack+aware+task+assignment+in+Kafka+Streams) and [**Kafka Streams Developer Guide**](/37/streams/developer-guide/config-streams.html#rack-aware-assignment-strategy) for more details. + * To account for a break in compatibility introduced in version 3.1.0, MirrorMaker 2 has added a new `replication.policy.internal.topic.separator.enabled` property. If upgrading from 3.0.x or earlier, it may be necessary to set this property to `false`; see the property's documentation for more details. + * Early access of tiered storage feature is available, and it is not recommended for use in production environments. Welcome to test it and provide any feedback to us. For more information about the early access tiered storage feature, please check [KIP-405](https://cwiki.apache.org/confluence/display/KAFKA/KIP-405%3A+Kafka+Tiered+Storage) and [Tiered Storage Early Access Release Note](https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Tiered+Storage+Early+Access+Release+Notes). + * Transaction partition verification ([KIP-890](https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense)) has been added to data partitions to prevent hanging transactions. This feature is enabled by default and can be disabled by setting `transaction.partition.verification.enable` to false. The configuration can also be updated dynamically and is applied to the broker. Workloads running on version 3.6.0 with compression can experience InvalidRecordExceptions and UnknownServerExceptions. Upgrading to 3.6.1 or newer or disabling the feature fixes the issue. + * The `ByteBufferDeserializer` ([KIP-863](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=225152035)) was changed to avoid unnecessary deep-copies of data. This changes the behavior of the deserializer as user code cannot make any assumptions about buffer position, limit, capacity any longer. + + + +## Upgrading to 3.5.2 from any version 0.8.x through 3.4.x + +All upgrade steps remain same as upgrading to 3.5.0 + +### Notable changes in 3.5.2 + + * When migrating producer ID blocks from ZK to KRaft, there could be duplicate producer IDs being given to transactional or idempotent producers. This can cause long term problems since the producer IDs are persisted and reused for a long time. See [KAFKA-15552](https://issues.apache.org/jira/browse/KAFKA-15552) for more details. + * In 3.5.0 and 3.5.1, there could be an issue that the empty ISR is returned from controller after AlterPartition request during rolling upgrade. This issue will impact the availability of the topic partition. See [KAFKA-15353](https://issues.apache.org/jira/browse/KAFKA-15353) for more details. + + + +## Upgrading to 3.5.1 from any version 0.8.x through 3.4.x + +All upgrade steps remain same as upgrading to 3.5.0 + +### Notable changes in 3.5.1 + + * Upgraded the dependency, snappy-java, to a version which is not vulnerable to [CVE-2023-34455.](https://nvd.nist.gov/vuln/detail/CVE-2023-34455) You can find more information about the CVE at [Kafka CVE list.](/community/cve-list/#CVE-2023-34455) + * Fixed a regression introduced in 3.3.0, which caused `security.protocol` configuration values to be restricted to upper case only. After the fix, `security.protocol` values are case insensitive. See [KAFKA-15053](https://issues.apache.org/jira/browse/KAFKA-15053) for details. + + + +## Upgrading to 3.5.0 from any version 0.8.x through 3.4.x + +### Upgrading ZooKeeper-based clusters + +**If you are upgrading from a version prior to 2.1.x, please see the note in step 5 below about the change to the schema used to store consumer offsets. Once you have changed the inter.broker.protocol.version to the latest version, it will not be possible to downgrade to a version prior to 2.1.** + +**For a rolling upgrade:** + + 1. Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. `3.4`, `3.3`, etc.) + * log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact following the upgrade for the details on what this configuration does.) +If you are upgrading from version 0.11.0.x or above, and you have not overridden the message format, then you only need to override the inter-broker protocol version. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. `3.4`, `3.3`, etc.) + 2. Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. Once you have done so, the brokers will be running the latest version and you can verify that the cluster's behavior and performance meets expectations. It is still possible to downgrade at this point if there are any problems. + 3. Once the cluster's behavior and performance has been verified, bump the protocol version by editing `inter.broker.protocol.version` and setting it to `3.5`. + 4. Restart the brokers one by one for the new protocol version to take effect. Once the brokers begin using the latest protocol version, it will no longer be possible to downgrade the cluster to an older version. + 5. If you have overridden the message format version as instructed above, then you need to do one more rolling restart to upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later, change log.message.format.version to 3.5 on each broker and restart them one by one. Note that the older Scala clients, which are no longer maintained, do not support the message format introduced in 0.11, so to avoid conversion costs (or to take advantage of exactly once semantics), the newer Java clients must be used. + + + +### Upgrading KRaft-based clusters + +**If you are upgrading from a version prior to 3.3.0, please see the note in step 3 below. Once you have changed the metadata.version to the latest version, it will not be possible to downgrade to a version prior to 3.3-IV0.** + +**For a rolling upgrade:** + + 1. Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. Once you have done so, the brokers will be running the latest version and you can verify that the cluster's behavior and performance meets expectations. + 2. Once the cluster's behavior and performance has been verified, bump the metadata.version by running ` ./bin/kafka-features.sh upgrade --metadata 3.5 ` + 3. Note that cluster metadata downgrade is not supported in this version since it has metadata changes. Every [MetadataVersion](https://github.com/apache/kafka/blob/trunk/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java) after 3.2.x has a boolean parameter that indicates if there are metadata changes (i.e. `IBP_3_3_IV3(7, "3.3", "IV3", true)` means this version has metadata changes). Given your current and target versions, a downgrade is only possible if there are no metadata changes in the versions between. + + + +### Notable changes in 3.5.0 + + * Kafka Streams has introduced a new state store type, versioned key-value stores, for storing multiple record versions per key, thereby enabling timestamped retrieval operations to return the latest record (per key) as of a specified timestamp. See [KIP-889](https://cwiki.apache.org/confluence/display/KAFKA/KIP-889%3A+Versioned+State+Stores) and [KIP-914](https://cwiki.apache.org/confluence/display/KAFKA/KIP-914%3A+DSL+Processor+Semantics+for+Versioned+Stores) for more details. If the new store typed is used in the DSL, improved processing semantics are applied as described in [KIP-914](https://cwiki.apache.org/confluence/display/KAFKA/KIP-914%3A+DSL+Processor+Semantics+for+Versioned+Stores). + * KTable aggregation semantics got further improved via [KIP-904](https://cwiki.apache.org/confluence/display/KAFKA/KIP-904%3A+Kafka+Streams+-+Guarantee+subtractor+is+called+before+adder+if+key+has+not+changed), now avoiding spurious intermediate results. + * Kafka Streams' `ProductionExceptionHandler` is improved via [KIP-399](https://cwiki.apache.org/confluence/display/KAFKA/KIP-399%3A+Extend+ProductionExceptionHandler+to+cover+serialization+exceptions), now also covering serialization errors. + * MirrorMaker now uses incrementalAlterConfigs API by default to synchronize topic configurations instead of the deprecated alterConfigs API. A new settings called `use.incremental.alter.configs` is introduced to allow users to control which API to use. This new setting is marked deprecated and will be removed in the next major release when incrementalAlterConfigs API is always used. See [KIP-894](https://cwiki.apache.org/confluence/display/KAFKA/KIP-894%3A+Use+incrementalAlterConfigs+API+for+syncing+topic+configurations) for more details. + * The JmxTool, EndToEndLatency, StreamsResetter, ConsumerPerformance and ClusterTool have been migrated to the tools module. The 'kafka.tools' package is deprecated and will change to 'org.apache.kafka.tools' in the next major release. See [KAFKA-14525](https://issues.apache.org/jira/browse/KAFKA-14525) for more details. + + + +## Upgrading to 3.4.0 from any version 0.8.x through 3.3.x + +**If you are upgrading from a version prior to 2.1.x, please see the note below about the change to the schema used to store consumer offsets. Once you have changed the inter.broker.protocol.version to the latest version, it will not be possible to downgrade to a version prior to 2.1.** + +**For a rolling upgrade:** + + 1. Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. `3.3`, `3.2`, etc.) + * log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact following the upgrade for the details on what this configuration does.) +If you are upgrading from version 0.11.0.x or above, and you have not overridden the message format, then you only need to override the inter-broker protocol version. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. `3.3`, `3.2`, etc.) + 2. Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. Once you have done so, the brokers will be running the latest version and you can verify that the cluster's behavior and performance meets expectations. It is still possible to downgrade at this point if there are any problems. + 3. Once the cluster's behavior and performance has been verified, bump the protocol version by editing `inter.broker.protocol.version` and setting it to `3.4`. + 4. Restart the brokers one by one for the new protocol version to take effect. Once the brokers begin using the latest protocol version, it will no longer be possible to downgrade the cluster to an older version. + 5. If you have overridden the message format version as instructed above, then you need to do one more rolling restart to upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later, change log.message.format.version to 3.4 on each broker and restart them one by one. Note that the older Scala clients, which are no longer maintained, do not support the message format introduced in 0.11, so to avoid conversion costs (or to take advantage of exactly once semantics), the newer Java clients must be used. + + + +## Upgrading a KRaft-based cluster to 3.4.0 from any version 3.0.x through 3.3.x + +**If you are upgrading from a version prior to 3.3.0, please see the note below. Once you have changed the metadata.version to the latest version, it will not be possible to downgrade to a version prior to 3.3-IV0.** + +**For a rolling upgrade:** + + 1. Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. Once you have done so, the brokers will be running the latest version and you can verify that the cluster's behavior and performance meets expectations. + 2. Once the cluster's behavior and performance has been verified, bump the metadata.version by running ` ./bin/kafka-features.sh upgrade --metadata 3.4 ` + 3. Note that cluster metadata downgrade is not supported in this version since it has metadata changes. Every [MetadataVersion](https://github.com/apache/kafka/blob/trunk/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java) after 3.2.x has a boolean parameter that indicates if there are metadata changes (i.e. `IBP_3_3_IV3(7, "3.3", "IV3", true)` means this version has metadata changes). Given your current and target versions, a downgrade is only possible if there are no metadata changes in the versions between. + + + +### Notable changes in 3.4.0 + + * Since Apache Kafka 3.4.0, we have added a system property ("org.apache.kafka.disallowed.login.modules") to disable the problematic login modules usage in SASL JAAS configuration. Also by default "com.sun.security.auth.module.JndiLoginModule" is disabled from Apache Kafka 3.4.0. + + + +## Upgrading to 3.3.1 from any version 0.8.x through 3.2.x + +**If you are upgrading from a version prior to 2.1.x, please see the note below about the change to the schema used to store consumer offsets. Once you have changed the inter.broker.protocol.version to the latest version, it will not be possible to downgrade to a version prior to 2.1.** + +**For a rolling upgrade:** + + 1. Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. `3.2`, `3.1`, etc.) + * log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact following the upgrade for the details on what this configuration does.) +If you are upgrading from version 0.11.0.x or above, and you have not overridden the message format, then you only need to override the inter-broker protocol version. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. `3.2`, `3.1`, etc.) + 2. Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. Once you have done so, the brokers will be running the latest version and you can verify that the cluster's behavior and performance meets expectations. It is still possible to downgrade at this point if there are any problems. + 3. Once the cluster's behavior and performance has been verified, bump the protocol version by editing `inter.broker.protocol.version` and setting it to `3.3`. + 4. Restart the brokers one by one for the new protocol version to take effect. Once the brokers begin using the latest protocol version, it will no longer be possible to downgrade the cluster to an older version. + 5. If you have overridden the message format version as instructed above, then you need to do one more rolling restart to upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later, change log.message.format.version to 3.3 on each broker and restart them one by one. Note that the older Scala clients, which are no longer maintained, do not support the message format introduced in 0.11, so to avoid conversion costs (or to take advantage of exactly once semantics), the newer Java clients must be used. + + + +## Upgrading a KRaft-based cluster to 3.3.1 from any version 3.0.x through 3.2.x + +**If you are upgrading from a version prior to 3.3.1, please see the note below. Once you have changed the metadata.version to the latest version, it will not be possible to downgrade to a version prior to 3.3-IV0.** + +**For a rolling upgrade:** + + 1. Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. Once you have done so, the brokers will be running the latest version and you can verify that the cluster's behavior and performance meets expectations. + 2. Once the cluster's behavior and performance has been verified, bump the metadata.version by running ` ./bin/kafka-features.sh upgrade --metadata 3.3 ` + 3. Note that cluster metadata downgrade is not supported in this version since it has metadata changes. Every [MetadataVersion](https://github.com/apache/kafka/blob/trunk/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java) after 3.2.x has a boolean parameter that indicates if there are metadata changes (i.e. `IBP_3_3_IV3(7, "3.3", "IV3", true)` means this version has metadata changes). Given your current and target versions, a downgrade is only possible if there are no metadata changes in the versions between. + + + +### Notable changes in 3.3.1 + + * KRaft mode is production ready for new clusters. See [KIP-833](https://cwiki.apache.org/confluence/display/KAFKA/KIP-833%3A+Mark+KRaft+as+Production+Ready) for more details (including limitations). + * The partitioner used by default for records with no keys has been improved to avoid pathological behavior when one or more brokers are slow. The new logic may affect the batching behavior, which can be tuned using the `batch.size` and/or `linger.ms` configuration settings. The previous behavior can be restored by setting `partitioner.class=org.apache.kafka.clients.producer.internals.DefaultPartitioner`. See [KIP-794](https://cwiki.apache.org/confluence/display/KAFKA/KIP-794%3A+Strictly+Uniform+Sticky+Partitioner) for more details. + * There is now a slightly different upgrade process for KRaft clusters than for ZK-based clusters, as described above. + * Introduced a new API `addMetricIfAbsent` to `Metrics` which would create a new Metric if not existing or return the same metric if already registered. Note that this behaviour is different from `addMetric` API which throws an `IllegalArgumentException` when trying to create an already existing metric. (See [KIP-843](https://cwiki.apache.org/confluence/display/KAFKA/KIP-843%3A+Adding+addMetricIfAbsent+method+to+Metrics) for more details). + + + +## Upgrading to 3.2.0 from any version 0.8.x through 3.1.x + +**If you are upgrading from a version prior to 2.1.x, please see the note below about the change to the schema used to store consumer offsets. Once you have changed the inter.broker.protocol.version to the latest version, it will not be possible to downgrade to a version prior to 2.1.** + +**For a rolling upgrade:** + + 1. Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. `3.1`, `3.0`, etc.) + * log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact following the upgrade for the details on what this configuration does.) +If you are upgrading from version 0.11.0.x or above, and you have not overridden the message format, then you only need to override the inter-broker protocol version. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. `3.1`, `3.0`, etc.) + 2. Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. Once you have done so, the brokers will be running the latest version and you can verify that the cluster's behavior and performance meets expectations. It is still possible to downgrade at this point if there are any problems. + 3. Once the cluster's behavior and performance has been verified, bump the protocol version by editing `inter.broker.protocol.version` and setting it to `3.2`. + 4. Restart the brokers one by one for the new protocol version to take effect. Once the brokers begin using the latest protocol version, it will no longer be possible to downgrade the cluster to an older version. + 5. If you have overridden the message format version as instructed above, then you need to do one more rolling restart to upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later, change log.message.format.version to 3.2 on each broker and restart them one by one. Note that the older Scala clients, which are no longer maintained, do not support the message format introduced in 0.11, so to avoid conversion costs (or to take advantage of exactly once semantics), the newer Java clients must be used. + + + +### Notable changes in 3.2.0 + + * Idempotence for the producer is enabled by default if no conflicting configurations are set. When producing to brokers older than 2.8.0, the `IDEMPOTENT_WRITE` permission is required. Check the compatibility section of [KIP-679](https://cwiki.apache.org/confluence/display/KAFKA/KIP-679%3A+Producer+will+enable+the+strongest+delivery+guarantee+by+default#KIP679:Producerwillenablethestrongestdeliveryguaranteebydefault-Compatibility,Deprecation,andMigrationPlan) for details. In 3.0.0 and 3.1.0, a bug prevented this default from being applied, which meant that idempotence remained disabled unless the user had explicitly set `enable.idempotence` to true (See [KAFKA-13598](https://issues.apache.org/jira/browse/KAFKA-13598) for more details). This issue was fixed and the default is properly applied in 3.0.1, 3.1.1, and 3.2.0. + * A notable exception is Connect that by default disables idempotent behavior for all of its producers in order to uniformly support using a wide range of Kafka broker versions. Users can change this behavior to enable idempotence for some or all producers via Connect worker and/or connector configuration. Connect may enable idempotent producers by default in a future major release. + * Kafka has replaced log4j with reload4j due to security concerns. This only affects modules that specify a logging backend (`connect-runtime` and `kafka-tools` are two such examples). A number of modules, including `kafka-clients`, leave it to the application to specify the logging backend. More information can be found at [reload4j](https://reload4j.qos.ch). Projects that depend on the affected modules from the Kafka project should use [slf4j-log4j12 version 1.7.35 or above](https://www.slf4j.org/manual.html#swapping) or slf4j-reload4j to avoid [possible compatibility issues originating from the logging framework](https://www.slf4j.org/codes.html#no_tlm). + * The example connectors, `FileStreamSourceConnector` and `FileStreamSinkConnector`, have been removed from the default classpath. To use them in Kafka Connect standalone or distributed mode they need to be explicitly added, for example `CLASSPATH=./libs/connect-file-3.2.0.jar ./bin/connect-distributed.sh`. + + + +## Upgrading to 3.1.0 from any version 0.8.x through 3.0.x + +**If you are upgrading from a version prior to 2.1.x, please see the note below about the change to the schema used to store consumer offsets. Once you have changed the inter.broker.protocol.version to the latest version, it will not be possible to downgrade to a version prior to 2.1.** + +**For a rolling upgrade:** + + 1. Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. `3.0`, `2.8`, etc.) + * log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact following the upgrade for the details on what this configuration does.) +If you are upgrading from version 0.11.0.x or above, and you have not overridden the message format, then you only need to override the inter-broker protocol version. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. `3.0`, `2.8`, etc.) + 2. Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. Once you have done so, the brokers will be running the latest version and you can verify that the cluster's behavior and performance meets expectations. It is still possible to downgrade at this point if there are any problems. + 3. Once the cluster's behavior and performance has been verified, bump the protocol version by editing `inter.broker.protocol.version` and setting it to `3.1`. + 4. Restart the brokers one by one for the new protocol version to take effect. Once the brokers begin using the latest protocol version, it will no longer be possible to downgrade the cluster to an older version. + 5. If you have overridden the message format version as instructed above, then you need to do one more rolling restart to upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later, change log.message.format.version to 3.1 on each broker and restart them one by one. Note that the older Scala clients, which are no longer maintained, do not support the message format introduced in 0.11, so to avoid conversion costs (or to take advantage of exactly once semantics), the newer Java clients must be used. + + + +### Notable changes in 3.1.1 + + * Idempotence for the producer is enabled by default if no conflicting configurations are set. When producing to brokers older than 2.8.0, the `IDEMPOTENT_WRITE` permission is required. Check the compatibility section of [KIP-679](https://cwiki.apache.org/confluence/display/KAFKA/KIP-679%3A+Producer+will+enable+the+strongest+delivery+guarantee+by+default#KIP679:Producerwillenablethestrongestdeliveryguaranteebydefault-Compatibility,Deprecation,andMigrationPlan) for details. A bug prevented the producer idempotence default from being applied which meant that it remained disabled unless the user had explicitly set `enable.idempotence` to true. See [KAFKA-13598](https://issues.apache.org/jira/browse/KAFKA-13598) for more details. This issue was fixed and the default is properly applied. + * A notable exception is Connect that by default disables idempotent behavior for all of its producers in order to uniformly support using a wide range of Kafka broker versions. Users can change this behavior to enable idempotence for some or all producers via Connect worker and/or connector configuration. Connect may enable idempotent producers by default in a future major release. + * Kafka has replaced log4j with reload4j due to security concerns. This only affects modules that specify a logging backend (`connect-runtime` and `kafka-tools` are two such examples). A number of modules, including `kafka-clients`, leave it to the application to specify the logging backend. More information can be found at [reload4j](https://reload4j.qos.ch). Projects that depend on the affected modules from the Kafka project should use [slf4j-log4j12 version 1.7.35 or above](https://www.slf4j.org/manual.html#swapping) or slf4j-reload4j to avoid [possible compatibility issues originating from the logging framework](https://www.slf4j.org/codes.html#no_tlm). + + + +### Notable changes in 3.1.0 + + * Apache Kafka supports Java 17. + * The following metrics have been deprecated: `bufferpool-wait-time-total`, `io-waittime-total`, and `iotime-total`. Please use `bufferpool-wait-time-ns-total`, `io-wait-time-ns-total`, and `io-time-ns-total` instead. See [KIP-773](https://cwiki.apache.org/confluence/display/KAFKA/KIP-773%3A+Differentiate+consistently+metric+latency+measured+in+millis+and+nanos) for more details. + * IBP 3.1 introduces topic IDs to FetchRequest as a part of [KIP-516](https://cwiki.apache.org/confluence/display/KAFKA/KIP-516%3A+Topic+Identifiers). + + + +## Upgrading to 3.0.1 from any version 0.8.x through 2.8.x + +**If you are upgrading from a version prior to 2.1.x, please see the note below about the change to the schema used to store consumer offsets. Once you have changed the inter.broker.protocol.version to the latest version, it will not be possible to downgrade to a version prior to 2.1.** + +**For a rolling upgrade:** + + 1. Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. `2.8`, `2.7`, etc.) + * log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact following the upgrade for the details on what this configuration does.) +If you are upgrading from version 0.11.0.x or above, and you have not overridden the message format, then you only need to override the inter-broker protocol version. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. `2.8`, `2.7`, etc.) + 2. Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. Once you have done so, the brokers will be running the latest version and you can verify that the cluster's behavior and performance meets expectations. It is still possible to downgrade at this point if there are any problems. + 3. Once the cluster's behavior and performance has been verified, bump the protocol version by editing `inter.broker.protocol.version` and setting it to `3.0`. + 4. Restart the brokers one by one for the new protocol version to take effect. Once the brokers begin using the latest protocol version, it will no longer be possible to downgrade the cluster to an older version. + 5. If you have overridden the message format version as instructed above, then you need to do one more rolling restart to upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later, change log.message.format.version to 3.0 on each broker and restart them one by one. Note that the older Scala clients, which are no longer maintained, do not support the message format introduced in 0.11, so to avoid conversion costs (or to take advantage of exactly once semantics), the newer Java clients must be used. + + + +### Notable changes in 3.0.1 + + * Idempotence for the producer is enabled by default if no conflicting configurations are set. When producing to brokers older than 2.8.0, the `IDEMPOTENT_WRITE` permission is required. Check the compatibility section of [KIP-679](https://cwiki.apache.org/confluence/display/KAFKA/KIP-679%3A+Producer+will+enable+the+strongest+delivery+guarantee+by+default#KIP679:Producerwillenablethestrongestdeliveryguaranteebydefault-Compatibility,Deprecation,andMigrationPlan) for details. A bug prevented the producer idempotence default from being applied which meant that it remained disabled unless the user had explicitly set `enable.idempotence` to true. See [KAFKA-13598](https://issues.apache.org/jira/browse/KAFKA-13598) for more details. This issue was fixed and the default is properly applied. + + + +### Notable changes in 3.0.0 + + * The producer has stronger delivery guarantees by default: `idempotence` is enabled and `acks` is set to `all` instead of `1`. See [KIP-679](https://cwiki.apache.org/confluence/display/KAFKA/KIP-679%3A+Producer+will+enable+the+strongest+delivery+guarantee+by+default) for details. In 3.0.0 and 3.1.0, a bug prevented the idempotence default from being applied which meant that it remained disabled unless the user had explicitly set `enable.idempotence` to true. Note that the bug did not affect the `acks=all` change. See [KAFKA-13598](https://issues.apache.org/jira/browse/KAFKA-13598) for more details. This issue was fixed and the default is properly applied in 3.0.1, 3.1.1, and 3.2.0. + * Java 8 and Scala 2.12 support have been deprecated since Apache Kafka 3.0 and will be removed in Apache Kafka 4.0. See [KIP-750](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=181308223) and [KIP-751](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=181308218) for more details. + * ZooKeeper has been upgraded to version 3.6.3. + * A preview of KRaft mode is available, though upgrading to it from the 2.8 Early Access release is not possible. See the KRaft section for details. + * The release tarball no longer includes test, sources, javadoc and test sources jars. These are still published to the Maven Central repository. + * A number of implementation dependency jars are [now available in the runtime classpath instead of compile and runtime classpaths](https://github.com/apache/kafka/pull/10203). Compilation errors after the upgrade can be fixed by adding the missing dependency jar(s) explicitly or updating the application not to use internal classes. + * The default value for the consumer configuration `session.timeout.ms` was increased from 10s to 45s. See [KIP-735](https://cwiki.apache.org/confluence/display/KAFKA/KIP-735%3A+Increase+default+consumer+session+timeout) for more details. + * The broker configuration `log.message.format.version` and topic configuration `message.format.version` have been deprecated. The value of both configurations is always assumed to be `3.0` if `inter.broker.protocol.version` is `3.0` or higher. If `log.message.format.version` or `message.format.version` are set, we recommend clearing them at the same time as the `inter.broker.protocol.version` upgrade to 3.0. This will avoid potential compatibility issues if the `inter.broker.protocol.version` is downgraded. See [KIP-724](https://cwiki.apache.org/confluence/display/KAFKA/KIP-724%3A+Drop+support+for+message+formats+v0+and+v1) for more details. + * The Streams API removed all deprecated APIs that were deprecated in version 2.5.0 or earlier. For a complete list of removed APIs compare the detailed Kafka Streams upgrade notes. + * Kafka Streams no longer has a compile time dependency on "connect:json" module ([KAFKA-5146](https://issues.apache.org/jira/browse/KAFKA-5146)). Projects that were relying on this transitive dependency will have to explicitly declare it. + * Custom principal builder implementations specified through `principal.builder.class` must now implement the `KafkaPrincipalSerde` interface to allow for forwarding between brokers. See [KIP-590](https://cwiki.apache.org/confluence/display/KAFKA/KIP-590%3A+Redirect+Zookeeper+Mutation+Protocols+to+The+Controller) for more details about the usage of KafkaPrincipalSerde. + * A number of deprecated classes, methods and tools have been removed from the `clients`, `connect`, `core` and `tools` modules: + * The Scala `Authorizer`, `SimpleAclAuthorizer` and related classes have been removed. Please use the Java `Authorizer` and `AclAuthorizer` instead. + * The `Metric#value()` method was removed ([KAFKA-12573](https://issues.apache.org/jira/browse/KAFKA-12573)). + * The `Sum` and `Total` classes were removed ([KAFKA-12584](https://issues.apache.org/jira/browse/KAFKA-12584)). Please use `WindowedSum` and `CumulativeSum` instead. + * The `Count` and `SampledTotal` classes were removed. Please use `WindowedCount` and `WindowedSum` respectively instead. + * The `PrincipalBuilder`, `DefaultPrincipalBuilder` and `ResourceFilter` classes were removed. + * Various constants and constructors were removed from `SslConfigs`, `SaslConfigs`, `AclBinding` and `AclBindingFilter`. + * The `Admin.electedPreferredLeaders()` methods were removed. Please use `Admin.electLeaders` instead. + * The `kafka-preferred-replica-election` command line tool was removed. Please use `kafka-leader-election` instead. + * The `--zookeeper` option was removed from the `kafka-topics` and `kafka-reassign-partitions` command line tools. Please use `--bootstrap-server` instead. + * In the `kafka-configs` command line tool, the `--zookeeper` option is only supported for updating SCRAM Credentials configuration and describing/updating dynamic broker configs when brokers are not running. Please use `--bootstrap-server` for other configuration operations. + * The `ConfigEntry` constructor was removed ([KAFKA-12577](https://issues.apache.org/jira/browse/KAFKA-12577)). Please use the remaining public constructor instead. + * The config value `default` for the client config `client.dns.lookup` has been removed. In the unlikely event that you set this config explicitly, we recommend leaving the config unset (`use_all_dns_ips` is used by default). + * The `ExtendedDeserializer` and `ExtendedSerializer` classes have been removed. Please use `Deserializer` and `Serializer` instead. + * The `close(long, TimeUnit)` method was removed from the producer, consumer and admin client. Please use `close(Duration)`. + * The `ConsumerConfig.addDeserializerToConfig` and `ProducerConfig.addSerializerToConfig` methods were removed. These methods were not intended to be public API and there is no replacement. + * The `NoOffsetForPartitionException.partition()` method was removed. Please use `partitions()` instead. + * The default `partition.assignment.strategy` is changed to "[RangeAssignor, CooperativeStickyAssignor]", which will use the RangeAssignor by default, but allows upgrading to the CooperativeStickyAssignor with just a single rolling bounce that removes the RangeAssignor from the list. Please check the client upgrade path guide [here](https://cwiki.apache.org/confluence/display/KAFKA/KIP-429:+Kafka+Consumer+Incremental+Rebalance+Protocol#KIP429:KafkaConsumerIncrementalRebalanceProtocol-Consumer) for more detail. + * The Scala `kafka.common.MessageFormatter` was removed. Please use the Java `org.apache.kafka.common.MessageFormatter`. + * The `MessageFormatter.init(Properties)` method was removed. Please use `configure(Map)` instead. + * The `checksum()` method has been removed from `ConsumerRecord` and `RecordMetadata`. The message format v2, which has been the default since 0.11, moved the checksum from the record to the record batch. As such, these methods don't make sense and no replacements exist. + * The `ChecksumMessageFormatter` class was removed. It is not part of the public API, but it may have been used with `kafka-console-consumer.sh`. It reported the checksum of each record, which has not been supported since message format v2. + * The `org.apache.kafka.clients.consumer.internals.PartitionAssignor` class has been removed. Please use `org.apache.kafka.clients.consumer.ConsumerPartitionAssignor` instead. + * The `quota.producer.default` and `quota.consumer.default` configurations were removed ([KAFKA-12591](https://issues.apache.org/jira/browse/KAFKA-12591)). Dynamic quota defaults must be used instead. + * The `port` and `host.name` configurations were removed. Please use `listeners` instead. + * The `advertised.port` and `advertised.host.name` configurations were removed. Please use `advertised.listeners` instead. + * The deprecated worker configurations `rest.host.name` and `rest.port` were removed ([KAFKA-12482](https://issues.apache.org/jira/browse/KAFKA-12482)) from the Kafka Connect worker configuration. Please use `listeners` instead. + * The `Producer#sendOffsetsToTransaction(Map offsets, String consumerGroupId)` method has been deprecated. Please use `Producer#sendOffsetsToTransaction(Map offsets, ConsumerGroupMetadata metadata)` instead, where the `ConsumerGroupMetadata` can be retrieved via `KafkaConsumer#groupMetadata()` for stronger semantics. Note that the full set of consumer group metadata is only understood by brokers or version 2.5 or higher, so you must upgrade your kafka cluster to get the stronger semantics. Otherwise, you can just pass in `new ConsumerGroupMetadata(consumerGroupId)` to work with older brokers. See [KIP-732](https://cwiki.apache.org/confluence/x/zJONCg) for more details. + * The Connect `internal.key.converter` and `internal.value.converter` properties have been completely [removed](https://cwiki.apache.org/confluence/x/2YDOCg). The use of these Connect worker properties has been deprecated since version 2.0.0. Workers are now hardcoded to use the JSON converter with `schemas.enable` set to `false`. If your cluster has been using a different internal key or value converter, you can follow the migration steps outlined in [KIP-738](https://cwiki.apache.org/confluence/x/2YDOCg) to safely upgrade your Connect cluster to 3.0. + * The Connect-based MirrorMaker (MM2) includes changes to support `IdentityReplicationPolicy`, enabling replication without renaming topics. The existing `DefaultReplicationPolicy` is still used by default, but identity replication can be enabled via the `replication.policy` configuration property. This is especially useful for users migrating from the older MirrorMaker (MM1), or for use-cases with simple one-way replication topologies where topic renaming is undesirable. Note that `IdentityReplicationPolicy`, unlike `DefaultReplicationPolicy`, cannot prevent replication cycles based on topic names, so take care to avoid cycles when constructing your replication topology. + * The original MirrorMaker (MM1) and related classes have been deprecated. Please use the Connect-based MirrorMaker (MM2), as described in the [Geo-Replication section](/37/#georeplication). + + + +## Upgrading to 2.8.1 from any version 0.8.x through 2.7.x + +**If you are upgrading from a version prior to 2.1.x, please see the note below about the change to the schema used to store consumer offsets. Once you have changed the inter.broker.protocol.version to the latest version, it will not be possible to downgrade to a version prior to 2.1.** + +**For a rolling upgrade:** + + 1. Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. `2.7`, `2.6`, etc.) + * log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact following the upgrade for the details on what this configuration does.) +If you are upgrading from version 0.11.0.x or above, and you have not overridden the message format, then you only need to override the inter-broker protocol version. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. `2.7`, `2.6`, etc.) + 2. Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. Once you have done so, the brokers will be running the latest version and you can verify that the cluster's behavior and performance meets expectations. It is still possible to downgrade at this point if there are any problems. + 3. Once the cluster's behavior and performance has been verified, bump the protocol version by editing `inter.broker.protocol.version` and setting it to `2.8`. + 4. Restart the brokers one by one for the new protocol version to take effect. Once the brokers begin using the latest protocol version, it will no longer be possible to downgrade the cluster to an older version. + 5. If you have overridden the message format version as instructed above, then you need to do one more rolling restart to upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later, change log.message.format.version to 2.8 on each broker and restart them one by one. Note that the older Scala clients, which are no longer maintained, do not support the message format introduced in 0.11, so to avoid conversion costs (or to take advantage of exactly once semantics), the newer Java clients must be used. + + + +### Notable changes in 2.8.0 + + * The 2.8.0 release added a new method to the Authorizer Interface introduced in [KIP-679](https://cwiki.apache.org/confluence/display/KAFKA/KIP-679%3A+Producer+will+enable+the+strongest+delivery+guarantee+by+default). The motivation is to unblock our future plan to enable the strongest message delivery guarantee by default. Custom authorizer should consider providing a more efficient implementation that supports audit logging and any custom configs or access rules. + * IBP 2.8 introduces topic IDs to topics as a part of [KIP-516](https://cwiki.apache.org/confluence/display/KAFKA/KIP-516%3A+Topic+Identifiers). When using ZooKeeper, this information is stored in the TopicZNode. If the cluster is downgraded to a previous IBP or version, future topics will not get topic IDs and it is not guaranteed that topics will retain their topic IDs in ZooKeeper. This means that upon upgrading again, some topics or all topics will be assigned new IDs. + * Kafka Streams introduce a type-safe `split()` operator as a substitution for deprecated `KStream#branch()` method (cf. [KIP-418](https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream)). + + + +## Upgrading to 2.7.0 from any version 0.8.x through 2.6.x + +**If you are upgrading from a version prior to 2.1.x, please see the note below about the change to the schema used to store consumer offsets. Once you have changed the inter.broker.protocol.version to the latest version, it will not be possible to downgrade to a version prior to 2.1.** + +**For a rolling upgrade:** + + 1. Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. `2.6`, `2.5`, etc.) + * log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact following the upgrade for the details on what this configuration does.) +If you are upgrading from version 0.11.0.x or above, and you have not overridden the message format, then you only need to override the inter-broker protocol version. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. `2.6`, `2.5`, etc.) + 2. Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. Once you have done so, the brokers will be running the latest version and you can verify that the cluster's behavior and performance meets expectations. It is still possible to downgrade at this point if there are any problems. + 3. Once the cluster's behavior and performance has been verified, bump the protocol version by editing `inter.broker.protocol.version` and setting it to `2.7`. + 4. Restart the brokers one by one for the new protocol version to take effect. Once the brokers begin using the latest protocol version, it will no longer be possible to downgrade the cluster to an older version. + 5. If you have overridden the message format version as instructed above, then you need to do one more rolling restart to upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later, change log.message.format.version to 2.7 on each broker and restart them one by one. Note that the older Scala clients, which are no longer maintained, do not support the message format introduced in 0.11, so to avoid conversion costs (or to take advantage of exactly once semantics), the newer Java clients must be used. + + + +### Notable changes in 2.7.0 + + * The 2.7.0 release includes the core Raft implementation specified in [KIP-595](https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum). There is a separate "raft" module containing most of the logic. Until integration with the controller is complete, there is a standalone server that users can use for testing the performance of the Raft implementation. See the README.md in the raft module for details + * KIP-651 [adds support](https://cwiki.apache.org/confluence/display/KAFKA/KIP-651+-+Support+PEM+format+for+SSL+certificates+and+private+key) for using PEM files for key and trust stores. + * KIP-612 [adds support](https://cwiki.apache.org/confluence/display/KAFKA/KIP-612%3A+Ability+to+Limit+Connection+Creation+Rate+on+Brokers) for enforcing broker-wide and per-listener connection create rates. The 2.7.0 release contains the first part of KIP-612 with dynamic configuration coming in the 2.8.0 release. + * The ability to throttle topic and partition creations or topics deletions to prevent a cluster from being harmed via [KIP-599](https://cwiki.apache.org/confluence/display/KAFKA/KIP-599%3A+Throttle+Create+Topic%2C+Create+Partition+and+Delete+Topic+Operations) + * When new features become available in Kafka there are two main issues: + 1. How do Kafka clients become aware of broker capabilities? + 2. How does the broker decide which features to enable? +[KIP-584](https://cwiki.apache.org/confluence/display/KAFKA/KIP-584%3A+Versioning+scheme+for+features) provides a flexible and operationally easy solution for client discovery, feature gating and rolling upgrades using a single restart. + * The ability to print record offsets and headers with the `ConsoleConsumer` is now possible via [KIP-431](https://cwiki.apache.org/confluence/display/KAFKA/KIP-431%3A+Support+of+printing+additional+ConsumerRecord+fields+in+DefaultMessageFormatter) + * The addition of [KIP-554](https://cwiki.apache.org/confluence/display/KAFKA/KIP-554%3A+Add+Broker-side+SCRAM+Config+API) continues progress towards the goal of Zookeeper removal from Kafka. The addition of KIP-554 means you don't have to connect directly to ZooKeeper anymore for managing SCRAM credentials. + * Altering non-reconfigurable configs of existent listeners causes `InvalidRequestException`. By contrast, the previous (unintended) behavior would have caused the updated configuration to be persisted, but it wouldn't take effect until the broker was restarted. See [KAFKA-10479](https://github.com/apache/kafka/pull/9284) for more discussion. See `DynamicBrokerConfig.DynamicSecurityConfigs` and `SocketServer.ListenerReconfigurableConfigs` for the supported reconfigurable configs of existent listeners. + * Kafka Streams adds support for [Sliding Windows Aggregations](https://cwiki.apache.org/confluence/display/KAFKA/KIP-450%3A+Sliding+Window+Aggregations+in+the+DSL) in the KStreams DSL. + * Reverse iteration over state stores enabling more efficient most recent update searches with [KIP-617](https://cwiki.apache.org/confluence/display/KAFKA/KIP-617%3A+Allow+Kafka+Streams+State+Stores+to+be+iterated+backwards) + * End-to-End latency metrics in Kafka Steams see [KIP-613](https://cwiki.apache.org/confluence/display/KAFKA/KIP-613%3A+Add+end-to-end+latency+metrics+to+Streams) for more details + * Kafka Streams added metrics reporting default RocksDB properties with [KIP-607](https://cwiki.apache.org/confluence/display/KAFKA/KIP-607%3A+Add+Metrics+to+Kafka+Streams+to+Report+Properties+of+RocksDB) + * Better Scala implicit Serdes support from [KIP-616](https://cwiki.apache.org/confluence/display/KAFKA/KIP-616%3A+Rename+implicit+Serdes+instances+in+kafka-streams-scala) + + + +## Upgrading to 2.6.0 from any version 0.8.x through 2.5.x + +**If you are upgrading from a version prior to 2.1.x, please see the note below about the change to the schema used to store consumer offsets. Once you have changed the inter.broker.protocol.version to the latest version, it will not be possible to downgrade to a version prior to 2.1.** + +**For a rolling upgrade:** + + 1. Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. `2.5`, `2.4`, etc.) + * log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact following the upgrade for the details on what this configuration does.) +If you are upgrading from version 0.11.0.x or above, and you have not overridden the message format, then you only need to override the inter-broker protocol version. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. `2.5`, `2.4`, etc.) + 2. Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. Once you have done so, the brokers will be running the latest version and you can verify that the cluster's behavior and performance meets expectations. It is still possible to downgrade at this point if there are any problems. + 3. Once the cluster's behavior and performance has been verified, bump the protocol version by editing `inter.broker.protocol.version` and setting it to `2.6`. + 4. Restart the brokers one by one for the new protocol version to take effect. Once the brokers begin using the latest protocol version, it will no longer be possible to downgrade the cluster to an older version. + 5. If you have overridden the message format version as instructed above, then you need to do one more rolling restart to upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later, change log.message.format.version to 2.6 on each broker and restart them one by one. Note that the older Scala clients, which are no longer maintained, do not support the message format introduced in 0.11, so to avoid conversion costs (or to take advantage of exactly once semantics), the newer Java clients must be used. + + + +### Notable changes in 2.6.0 + + * Kafka Streams adds a new processing mode (requires broker 2.5 or newer) that improves application scalability using exactly-once guarantees (cf. [KIP-447](https://cwiki.apache.org/confluence/display/KAFKA/KIP-447%3A+Producer+scalability+for+exactly+once+semantics)) + * TLSv1.3 has been enabled by default for Java 11 or newer. The client and server will negotiate TLSv1.3 if both support it and fallback to TLSv1.2 otherwise. See [KIP-573](https://cwiki.apache.org/confluence/display/KAFKA/KIP-573%3A+Enable+TLSv1.3+by+default) for more details. + * The default value for the `client.dns.lookup` configuration has been changed from `default` to `use_all_dns_ips`. If a hostname resolves to multiple IP addresses, clients and brokers will now attempt to connect to each IP in sequence until the connection is successfully established. See [KIP-602](https://cwiki.apache.org/confluence/display/KAFKA/KIP-602%3A+Change+default+value+for+client.dns.lookup) for more details. + * `NotLeaderForPartitionException` has been deprecated and replaced with `NotLeaderOrFollowerException`. Fetch requests and other requests intended only for the leader or follower return NOT_LEADER_OR_FOLLOWER(6) instead of REPLICA_NOT_AVAILABLE(9) if the broker is not a replica, ensuring that this transient error during reassignments is handled by all clients as a retriable exception. + + + +## Upgrading to 2.5.0 from any version 0.8.x through 2.4.x + +**If you are upgrading from a version prior to 2.1.x, please see the note below about the change to the schema used to store consumer offsets. Once you have changed the inter.broker.protocol.version to the latest version, it will not be possible to downgrade to a version prior to 2.1.** + +**For a rolling upgrade:** + + 1. Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. `2.4`, `2.3`, etc.) + * log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact following the upgrade for the details on what this configuration does.) +If you are upgrading from version 0.11.0.x or above, and you have not overridden the message format, then you only need to override the inter-broker protocol version. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. `2.4`, `2.3`, etc.) + 2. Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. Once you have done so, the brokers will be running the latest version and you can verify that the cluster's behavior and performance meets expectations. It is still possible to downgrade at this point if there are any problems. + 3. Once the cluster's behavior and performance has been verified, bump the protocol version by editing `inter.broker.protocol.version` and setting it to `2.5`. + 4. Restart the brokers one by one for the new protocol version to take effect. Once the brokers begin using the latest protocol version, it will no longer be possible to downgrade the cluster to an older version. + 5. If you have overridden the message format version as instructed above, then you need to do one more rolling restart to upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later, change log.message.format.version to 2.5 on each broker and restart them one by one. Note that the older Scala clients, which are no longer maintained, do not support the message format introduced in 0.11, so to avoid conversion costs (or to take advantage of exactly once semantics), the newer Java clients must be used. + 6. There are several notable changes to the reassignment tool `kafka-reassign-partitions.sh` following the completion of [KIP-455](https://cwiki.apache.org/confluence/display/KAFKA/KIP-455%3A+Create+an+Administrative+API+for+Replica+Reassignment). This tool now requires the `--additional` flag to be provided when changing the throttle of an active reassignment. Reassignment cancellation is now possible using the `--cancel` command. Finally, reassignment with `--zookeeper` has been deprecated in favor of `--bootstrap-server`. See the KIP for more detail. + + + +### Notable changes in 2.5.0 + + * When `RebalanceProtocol#COOPERATIVE` is used, `Consumer#poll` can still return data while it is in the middle of a rebalance for those partitions still owned by the consumer; in addition `Consumer#commitSync` now may throw a non-fatal `RebalanceInProgressException` to notify users of such an event, in order to distinguish from the fatal `CommitFailedException` and allow users to complete the ongoing rebalance and then reattempt committing offsets for those still-owned partitions. + * For improved resiliency in typical network environments, the default value of `zookeeper.session.timeout.ms` has been increased from 6s to 18s and `replica.lag.time.max.ms` from 10s to 30s. + * New DSL operator `cogroup()` has been added for aggregating multiple streams together at once. + * Added a new `KStream.toTable()` API to translate an input event stream into a KTable. + * Added a new Serde type `Void` to represent null keys or null values from input topic. + * Deprecated `UsePreviousTimeOnInvalidTimestamp` and replaced it with `UsePartitionTimeOnInvalidTimeStamp`. + * Improved exactly-once semantics by adding a pending offset fencing mechanism and stronger transactional commit consistency check, which greatly simplifies the implementation of a scalable exactly-once application. We also added a new exactly-once semantics code example under [examples](https://github.com/apache/kafka/tree/2.5/examples) folder. Check out [KIP-447](https://cwiki.apache.org/confluence/display/KAFKA/KIP-447%3A+Producer+scalability+for+exactly+once+semantics) for the full details. + * Added a new public api `KafkaStreams.queryMetadataForKey(String, K, Serializer) to get detailed information on the key being queried. It provides information about the partition number where the key resides in addition to hosts containing the active and standby partitions for the key.` + * Provided support to query stale stores (for high availability) and the stores belonging to a specific partition by deprecating `KafkaStreams.store(String, QueryableStoreType)` and replacing it with `KafkaStreams.store(StoreQueryParameters)`. + * Added a new public api to access lag information for stores local to an instance with `KafkaStreams.allLocalStorePartitionLags()`. + * Scala 2.11 is no longer supported. See [KIP-531](https://cwiki.apache.org/confluence/display/KAFKA/KIP-531%3A+Drop+support+for+Scala+2.11+in+Kafka+2.5) for details. + * All Scala classes from the package `kafka.security.auth` have been deprecated. See [KIP-504](https://cwiki.apache.org/confluence/display/KAFKA/KIP-504+-+Add+new+Java+Authorizer+Interface) for details of the new Java authorizer API added in 2.4.0. Note that `kafka.security.auth.Authorizer` and `kafka.security.auth.SimpleAclAuthorizer` were deprecated in 2.4.0. + * TLSv1 and TLSv1.1 have been disabled by default since these have known security vulnerabilities. Only TLSv1.2 is now enabled by default. You can continue to use TLSv1 and TLSv1.1 by explicitly enabling these in the configuration options `ssl.protocol` and `ssl.enabled.protocols`. + * ZooKeeper has been upgraded to 3.5.7, and a ZooKeeper upgrade from 3.4.X to 3.5.7 can fail if there are no snapshot files in the 3.4 data directory. This usually happens in test upgrades where ZooKeeper 3.5.7 is trying to load an existing 3.4 data dir in which no snapshot file has been created. For more details about the issue please refer to [ZOOKEEPER-3056](https://issues.apache.org/jira/browse/ZOOKEEPER-3056). A fix is given in [ZOOKEEPER-3056](https://issues.apache.org/jira/browse/ZOOKEEPER-3056), which is to set `snapshot.trust.empty=true` config in `zookeeper.properties` before the upgrade. + * ZooKeeper version 3.5.7 supports TLS-encrypted connectivity to ZooKeeper both with or without client certificates, and additional Kafka configurations are available to take advantage of this. See [KIP-515](https://cwiki.apache.org/confluence/display/KAFKA/KIP-515%3A+Enable+ZK+client+to+use+the+new+TLS+supported+authentication) for details. + + + +## Upgrading from 0.8.x, 0.9.x, 0.10.0.x, 0.10.1.x, 0.10.2.x, 0.11.0.x, 1.0.x, 1.1.x, 2.0.x or 2.1.x or 2.2.x or 2.3.x to 2.4.0 + +**If you are upgrading from a version prior to 2.1.x, please see the note below about the change to the schema used to store consumer offsets. Once you have changed the inter.broker.protocol.version to the latest version, it will not be possible to downgrade to a version prior to 2.1.** + +**For a rolling upgrade:** + + 1. Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.10.0, 0.11.0, 1.0, 2.0, 2.2). + * log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact following the upgrade for the details on what this configuration does.) +If you are upgrading from version 0.11.0.x or above, and you have not overridden the message format, then you only need to override the inter-broker protocol version. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (0.11.0, 1.0, 1.1, 2.0, 2.1, 2.2, 2.3). + 2. Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. Once you have done so, the brokers will be running the latest version and you can verify that the cluster's behavior and performance meets expectations. It is still possible to downgrade at this point if there are any problems. + 3. Once the cluster's behavior and performance has been verified, bump the protocol version by editing `inter.broker.protocol.version` and setting it to 2.4. + 4. Restart the brokers one by one for the new protocol version to take effect. Once the brokers begin using the latest protocol version, it will no longer be possible to downgrade the cluster to an older version. + 5. If you have overridden the message format version as instructed above, then you need to do one more rolling restart to upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later, change log.message.format.version to 2.4 on each broker and restart them one by one. Note that the older Scala clients, which are no longer maintained, do not support the message format introduced in 0.11, so to avoid conversion costs (or to take advantage of exactly once semantics), the newer Java clients must be used. + + + +**Additional Upgrade Notes:** + + 1. ZooKeeper has been upgraded to 3.5.6. ZooKeeper upgrade from 3.4.X to 3.5.6 can fail if there are no snapshot files in 3.4 data directory. This usually happens in test upgrades where ZooKeeper 3.5.6 is trying to load an existing 3.4 data dir in which no snapshot file has been created. For more details about the issue please refer to [ZOOKEEPER-3056](https://issues.apache.org/jira/browse/ZOOKEEPER-3056). A fix is given in [ZOOKEEPER-3056](https://issues.apache.org/jira/browse/ZOOKEEPER-3056), which is to set `snapshot.trust.empty=true` config in `zookeeper.properties` before the upgrade. But we have observed data loss in standalone cluster upgrades when using `snapshot.trust.empty=true` config. For more details about the issue please refer to [ZOOKEEPER-3644](https://issues.apache.org/jira/browse/ZOOKEEPER-3644). So we recommend the safe workaround of copying empty [snapshot](https://issues.apache.org/jira/secure/attachment/12928686/snapshot.0) file to the 3.4 data directory, if there are no snapshot files in 3.4 data directory. For more details about the workaround please refer to [ZooKeeper Upgrade FAQ](https://cwiki.apache.org/confluence/display/ZOOKEEPER/Upgrade+FAQ). + 2. An embedded Jetty based [AdminServer](https://zookeeper.apache.org/doc/r3.5.6/zookeeperAdmin.html#sc_adminserver) added in ZooKeeper 3.5. AdminServer is enabled by default in ZooKeeper and is started on port 8080. AdminServer is disabled by default in the ZooKeeper config (`zookeeper.properties`) provided by the Apache Kafka distribution. Make sure to update your local `zookeeper.properties` file with `admin.enableServer=false` if you wish to disable the AdminServer. Please refer [AdminServer config](https://zookeeper.apache.org/doc/r3.5.6/zookeeperAdmin.html#sc_adminserver) to configure the AdminServer. + + + +### Notable changes in 2.4.0 + + * A new Admin API has been added for partition reassignments. Due to changing the way Kafka propagates reassignment information, it is possible to lose reassignment state in failure edge cases while upgrading to the new version. It is not recommended to start reassignments while upgrading. + * ZooKeeper has been upgraded from 3.4.14 to 3.5.6. TLS and dynamic reconfiguration are supported by the new version. + * The `bin/kafka-preferred-replica-election.sh` command line tool has been deprecated. It has been replaced by `bin/kafka-leader-election.sh`. + * The methods `electPreferredLeaders` in the Java `AdminClient` class have been deprecated in favor of the methods `electLeaders`. + * Scala code leveraging the `NewTopic(String, int, short)` constructor with literal values will need to explicitly call `toShort` on the second literal. + * The argument in the constructor `GroupAuthorizationException(String)` is now used to specify an exception message. Previously it referred to the group that failed authorization. This was done for consistency with other exception types and to avoid potential misuse. The constructor `TopicAuthorizationException(String)` which was previously used for a single unauthorized topic was changed similarly. + * The internal `PartitionAssignor` interface has been deprecated and replaced with a new `ConsumerPartitionAssignor` in the public API. Some methods/signatures are slightly different between the two interfaces. Users implementing a custom PartitionAssignor should migrate to the new interface as soon as possible. + * The `DefaultPartitioner` now uses a sticky partitioning strategy. This means that records for specific topic with null keys and no assigned partition will be sent to the same partition until the batch is ready to be sent. When a new batch is created, a new partition is chosen. This decreases latency to produce, but it may result in uneven distribution of records across partitions in edge cases. Generally users will not be impacted, but this difference may be noticeable in tests and other situations producing records for a very short amount of time. + * The blocking `KafkaConsumer#committed` methods have been extended to allow a list of partitions as input parameters rather than a single partition. It enables fewer request/response iterations between clients and brokers fetching for the committed offsets for the consumer group. The old overloaded functions are deprecated and we would recommend users to make their code changes to leverage the new methods (details can be found in [KIP-520](https://cwiki.apache.org/confluence/display/KAFKA/KIP-520%3A+Add+overloaded+Consumer%23committed+for+batching+partitions)). + * We've introduced a new `INVALID_RECORD` error in the produce response to distinguish from the `CORRUPT_MESSAGE` error. To be more concrete, previously when a batch of records was sent as part of a single request to the broker and one or more of the records failed the validation due to various causes (mismatch magic bytes, crc checksum errors, null key for log compacted topics, etc), the whole batch would be rejected with the same and misleading `CORRUPT_MESSAGE`, and the caller of the producer client would see the corresponding exception from either the future object of `RecordMetadata` returned from the `send` call as well as in the `Callback#onCompletion(RecordMetadata metadata, Exception exception)` Now with the new error code and improved error messages of the exception, producer callers would be better informed about the root cause why their sent records were failed. + * We are introducing incremental cooperative rebalancing to the clients' group protocol, which allows consumers to keep all of their assigned partitions during a rebalance and at the end revoke only those which must be migrated to another consumer for overall cluster balance. The `ConsumerCoordinator` will choose the latest `RebalanceProtocol` that is commonly supported by all of the consumer's supported assignors. You can use the new built-in `CooperativeStickyAssignor` or plug in your own custom cooperative assignor. To do so you must implement the `ConsumerPartitionAssignor` interface and include `RebalanceProtocol.COOPERATIVE` in the list returned by `ConsumerPartitionAssignor#supportedProtocols`. Your custom assignor can then leverage the `ownedPartitions` field in each consumer's `Subscription` to give partitions back to their previous owners whenever possible. Note that when a partition is to be reassigned to another consumer, it _must_ be removed from the new assignment until it has been revoked from its original owner. Any consumer that has to revoke a partition will trigger a followup rebalance to allow the revoked partition to safely be assigned to its new owner. See the [ConsumerPartitionAssignor RebalanceProtocol javadocs](/https://kafka.apache.org/24/javadoc/index.html?org/apache/kafka/clients/consumer/ConsumerPartitionAssignor.RebalanceProtocol.html) for more information. +To upgrade from the old (eager) protocol, which always revokes all partitions before rebalancing, to cooperative rebalancing, you must follow a specific upgrade path to get all clients on the same `ConsumerPartitionAssignor` that supports the cooperative protocol. This can be done with two rolling bounces, using the `CooperativeStickyAssignor` for the example: during the first one, add "cooperative-sticky" to the list of supported assignors for each member (without removing the previous assignor -- note that if previously using the default, you must include that explicitly as well). You then bounce and/or upgrade it. Once the entire group is on 2.4+ and all members have the "cooperative-sticky" among their supported assignors, remove the other assignor(s) and perform a second rolling bounce so that by the end all members support only the cooperative protocol. For further details on the cooperative rebalancing protocol and upgrade path, see [KIP-429](https://cwiki.apache.org/confluence/x/vAclBg). + * There are some behavioral changes to the `ConsumerRebalanceListener`, as well as a new API. Exceptions thrown during any of the listener's three callbacks will no longer be swallowed, and will instead be re-thrown all the way up to the `Consumer.poll()` call. The `onPartitionsLost` method has been added to allow users to react to abnormal circumstances where a consumer may have lost ownership of its partitions (such as a missed rebalance) and cannot commit offsets. By default, this will simply call the existing `onPartitionsRevoked` API to align with previous behavior. Note however that `onPartitionsLost` will not be called when the set of lost partitions is empty. This means that no callback will be invoked at the beginning of the first rebalance of a new consumer joining the group. +The semantics of the `ConsumerRebalanceListener's` callbacks are further changed when following the cooperative rebalancing protocol described above. In addition to `onPartitionsLost`, `onPartitionsRevoked` will also never be called when the set of revoked partitions is empty. The callback will generally be invoked only at the end of a rebalance, and only on the set of partitions that are being moved to another consumer. The `onPartitionsAssigned` callback will however always be called, even with an empty set of partitions, as a way to notify users of a rebalance event (this is true for both cooperative and eager). For details on the new callback semantics, see the [ConsumerRebalanceListener javadocs](/https://kafka.apache.org/24/javadoc/index.html?org/apache/kafka/clients/consumer/ConsumerRebalanceListener.html). + * The Scala trait `kafka.security.auth.Authorizer` has been deprecated and replaced with a new Java API `org.apache.kafka.server.authorizer.Authorizer`. The authorizer implementation class `kafka.security.auth.SimpleAclAuthorizer` has also been deprecated and replaced with a new implementation `kafka.security.authorizer.AclAuthorizer`. `AclAuthorizer` uses features supported by the new API to improve authorization logging and is compatible with `SimpleAclAuthorizer`. For more details, see [KIP-504](https://cwiki.apache.org/confluence/display/KAFKA/KIP-504+-+Add+new+Java+Authorizer+Interface). + + + +## Upgrading from 0.8.x, 0.9.x, 0.10.0.x, 0.10.1.x, 0.10.2.x, 0.11.0.x, 1.0.x, 1.1.x, 2.0.x or 2.1.x or 2.2.x to 2.3.0 + +**If you are upgrading from a version prior to 2.1.x, please see the note below about the change to the schema used to store consumer offsets. Once you have changed the inter.broker.protocol.version to the latest version, it will not be possible to downgrade to a version prior to 2.1.** + +**For a rolling upgrade:** + + 1. Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.8.2, 0.9.0, 0.10.0, 0.10.1, 0.10.2, 0.11.0, 1.0, 1.1). + * log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact following the upgrade for the details on what this configuration does.) +If you are upgrading from 0.11.0.x, 1.0.x, 1.1.x, 2.0.x, or 2.1.x, and you have not overridden the message format, then you only need to override the inter-broker protocol version. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (0.11.0, 1.0, 1.1, 2.0, 2.1, 2.2). + 2. Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. Once you have done so, the brokers will be running the latest version and you can verify that the cluster's behavior and performance meets expectations. It is still possible to downgrade at this point if there are any problems. + 3. Once the cluster's behavior and performance has been verified, bump the protocol version by editing `inter.broker.protocol.version` and setting it to 2.3. + 4. Restart the brokers one by one for the new protocol version to take effect. Once the brokers begin using the latest protocol version, it will no longer be possible to downgrade the cluster to an older version. + 5. If you have overridden the message format version as instructed above, then you need to do one more rolling restart to upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later, change log.message.format.version to 2.3 on each broker and restart them one by one. Note that the older Scala clients, which are no longer maintained, do not support the message format introduced in 0.11, so to avoid conversion costs (or to take advantage of exactly once semantics), the newer Java clients must be used. + + + +### Notable changes in 2.3.0 + + * We are introducing a new rebalancing protocol for Kafka Connect based on [incremental cooperative rebalancing](https://cwiki.apache.org/confluence/display/KAFKA/KIP-415%3A+Incremental+Cooperative+Rebalancing+in+Kafka+Connect). The new protocol does not require stopping all the tasks during a rebalancing phase between Connect workers. Instead, only the tasks that need to be exchanged between workers are stopped and they are started in a follow up rebalance. The new Connect protocol is enabled by default beginning with 2.3.0. For more details on how it works and how to enable the old behavior of eager rebalancing, checkout [incremental cooperative rebalancing design](/37/#connect_administration). + * We are introducing static membership towards consumer user. This feature reduces unnecessary rebalances during normal application upgrades or rolling bounces. For more details on how to use it, checkout [static membership design](/37/#static_membership). + * Kafka Streams DSL switches its used store types. While this change is mainly transparent to users, there are some corner cases that may require code changes. See the [Kafka Streams upgrade section](/37/streams/upgrade-guide#streams_api_changes_230) for more details. + * Kafka Streams 2.3.0 requires 0.11 message format or higher and does not work with older message format. + + + +## Upgrading from 0.8.x, 0.9.x, 0.10.0.x, 0.10.1.x, 0.10.2.x, 0.11.0.x, 1.0.x, 1.1.x, 2.0.x or 2.1.x to 2.2.0 + +**If you are upgrading from a version prior to 2.1.x, please see the note below about the change to the schema used to store consumer offsets. Once you have changed the inter.broker.protocol.version to the latest version, it will not be possible to downgrade to a version prior to 2.1.** + +**For a rolling upgrade:** + + 1. Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.8.2, 0.9.0, 0.10.0, 0.10.1, 0.10.2, 0.11.0, 1.0, 1.1). + * log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact following the upgrade for the details on what this configuration does.) +If you are upgrading from 0.11.0.x, 1.0.x, 1.1.x, or 2.0.x and you have not overridden the message format, then you only need to override the inter-broker protocol version. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (0.11.0, 1.0, 1.1, 2.0). + 2. Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. Once you have done so, the brokers will be running the latest version and you can verify that the cluster's behavior and performance meets expectations. It is still possible to downgrade at this point if there are any problems. + 3. Once the cluster's behavior and performance has been verified, bump the protocol version by editing `inter.broker.protocol.version` and setting it to 2.2. + 4. Restart the brokers one by one for the new protocol version to take effect. Once the brokers begin using the latest protocol version, it will no longer be possible to downgrade the cluster to an older version. + 5. If you have overridden the message format version as instructed above, then you need to do one more rolling restart to upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later, change log.message.format.version to 2.2 on each broker and restart them one by one. Note that the older Scala clients, which are no longer maintained, do not support the message format introduced in 0.11, so to avoid conversion costs (or to take advantage of exactly once semantics), the newer Java clients must be used. + + + +### Notable changes in 2.2.1 + + * Kafka Streams 2.2.1 requires 0.11 message format or higher and does not work with older message format. + + + +### Notable changes in 2.2.0 + + * The default consumer group id has been changed from the empty string (`""`) to `null`. Consumers who use the new default group id will not be able to subscribe to topics, and fetch or commit offsets. The empty string as consumer group id is deprecated but will be supported until a future major release. Old clients that rely on the empty string group id will now have to explicitly provide it as part of their consumer config. For more information see [KIP-289](https://cwiki.apache.org/confluence/display/KAFKA/KIP-289%3A+Improve+the+default+group+id+behavior+in+KafkaConsumer). + * The `bin/kafka-topics.sh` command line tool is now able to connect directly to brokers with `--bootstrap-server` instead of zookeeper. The old `--zookeeper` option is still available for now. Please read [KIP-377](https://cwiki.apache.org/confluence/display/KAFKA/KIP-377%3A+TopicCommand+to+use+AdminClient) for more information. + * Kafka Streams depends on a newer version of RocksDBs that requires MacOS 10.13 or higher. + + + +## Upgrading from 0.8.x, 0.9.x, 0.10.0.x, 0.10.1.x, 0.10.2.x, 0.11.0.x, 1.0.x, 1.1.x, or 2.0.0 to 2.1.0 + +**Note that 2.1.x contains a change to the internal schema used to store consumer offsets. Once the upgrade is complete, it will not be possible to downgrade to previous versions. See the rolling upgrade notes below for more detail.** + +**For a rolling upgrade:** + + 1. Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.8.2, 0.9.0, 0.10.0, 0.10.1, 0.10.2, 0.11.0, 1.0, 1.1). + * log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact following the upgrade for the details on what this configuration does.) +If you are upgrading from 0.11.0.x, 1.0.x, 1.1.x, or 2.0.x and you have not overridden the message format, then you only need to override the inter-broker protocol version. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (0.11.0, 1.0, 1.1, 2.0). + 2. Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. Once you have done so, the brokers will be running the latest version and you can verify that the cluster's behavior and performance meets expectations. It is still possible to downgrade at this point if there are any problems. + 3. Once the cluster's behavior and performance has been verified, bump the protocol version by editing `inter.broker.protocol.version` and setting it to 2.1. + 4. Restart the brokers one by one for the new protocol version to take effect. Once the brokers begin using the latest protocol version, it will no longer be possible to downgrade the cluster to an older version. + 5. If you have overridden the message format version as instructed above, then you need to do one more rolling restart to upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later, change log.message.format.version to 2.1 on each broker and restart them one by one. Note that the older Scala clients, which are no longer maintained, do not support the message format introduced in 0.11, so to avoid conversion costs (or to take advantage of exactly once semantics), the newer Java clients must be used. + + + +**Additional Upgrade Notes:** + + 1. Offset expiration semantics has slightly changed in this version. According to the new semantics, offsets of partitions in a group will not be removed while the group is subscribed to the corresponding topic and is still active (has active consumers). If group becomes empty all its offsets will be removed after default offset retention period (or the one set by broker) has passed (unless the group becomes active again). Offsets associated with standalone (simple) consumers, that do not use Kafka group management, will be removed after default offset retention period (or the one set by broker) has passed since their last commit. + 2. The default for console consumer's `enable.auto.commit` property when no `group.id` is provided is now set to `false`. This is to avoid polluting the consumer coordinator cache as the auto-generated group is not likely to be used by other consumers. + 3. The default value for the producer's `retries` config was changed to `Integer.MAX_VALUE`, as we introduced `delivery.timeout.ms` in [KIP-91](https://cwiki.apache.org/confluence/display/KAFKA/KIP-91+Provide+Intuitive+User+Timeouts+in+The+Producer), which sets an upper bound on the total time between sending a record and receiving acknowledgement from the broker. By default, the delivery timeout is set to 2 minutes. + 4. By default, MirrorMaker now overrides `delivery.timeout.ms` to `Integer.MAX_VALUE` when configuring the producer. If you have overridden the value of `retries` in order to fail faster, you will instead need to override `delivery.timeout.ms`. + 5. The `ListGroup` API now expects, as a recommended alternative, `Describe Group` access to the groups a user should be able to list. Even though the old `Describe Cluster` access is still supported for backward compatibility, using it for this API is not advised. + 6. [KIP-336](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=87298242) deprecates the ExtendedSerializer and ExtendedDeserializer interfaces and propagates the usage of Serializer and Deserializer. ExtendedSerializer and ExtendedDeserializer were introduced with [KIP-82](https://cwiki.apache.org/confluence/display/KAFKA/KIP-82+-+Add+Record+Headers) to provide record headers for serializers and deserializers in a Java 7 compatible fashion. Now we consolidated these interfaces as Java 7 support has been dropped since. + + + +### Notable changes in 2.1.0 + + * Jetty has been upgraded to 9.4.12, which excludes TLS_RSA_* ciphers by default because they do not support forward secrecy, see https://github.com/eclipse/jetty.project/issues/2807 for more information. + * Unclean leader election is automatically enabled by the controller when `unclean.leader.election.enable` config is dynamically updated by using per-topic config override. + * The `AdminClient` has added a method `AdminClient#metrics()`. Now any application using the `AdminClient` can gain more information and insight by viewing the metrics captured from the `AdminClient`. For more information see [KIP-324](https://cwiki.apache.org/confluence/display/KAFKA/KIP-324%3A+Add+method+to+get+metrics%28%29+in+AdminClient) + * Kafka now supports Zstandard compression from [KIP-110](https://cwiki.apache.org/confluence/display/KAFKA/KIP-110%3A+Add+Codec+for+ZStandard+Compression). You must upgrade the broker as well as clients to make use of it. Consumers prior to 2.1.0 will not be able to read from topics which use Zstandard compression, so you should not enable it for a topic until all downstream consumers are upgraded. See the KIP for more detail. + + + +## Upgrading from 0.8.x, 0.9.x, 0.10.0.x, 0.10.1.x, 0.10.2.x, 0.11.0.x, 1.0.x, or 1.1.x to 2.0.0 + +Kafka 2.0.0 introduces wire protocol changes. By following the recommended rolling upgrade plan below, you guarantee no downtime during the upgrade. However, please review the notable changes in 2.0.0 before upgrading. + +**For a rolling upgrade:** + + 1. Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.8.2, 0.9.0, 0.10.0, 0.10.1, 0.10.2, 0.11.0, 1.0, 1.1). + * log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact following the upgrade for the details on what this configuration does.) +If you are upgrading from 0.11.0.x, 1.0.x, or 1.1.x and you have not overridden the message format, then you only need to override the inter-broker protocol format. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (0.11.0, 1.0, 1.1). + 2. Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. + 3. Once the entire cluster is upgraded, bump the protocol version by editing `inter.broker.protocol.version` and setting it to 2.0. + 4. Restart the brokers one by one for the new protocol version to take effect. + 5. If you have overridden the message format version as instructed above, then you need to do one more rolling restart to upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later, change log.message.format.version to 2.0 on each broker and restart them one by one. Note that the older Scala consumer does not support the new message format introduced in 0.11, so to avoid the performance cost of down-conversion (or to take advantage of exactly once semantics), the newer Java consumer must be used. + + + +**Additional Upgrade Notes:** + + 1. If you are willing to accept downtime, you can simply take all the brokers down, update the code and start them back up. They will start with the new protocol by default. + 2. Bumping the protocol version and restarting can be done any time after the brokers are upgraded. It does not have to be immediately after. Similarly for the message format version. + 3. If you are using Java8 method references in your Kafka Streams code you might need to update your code to resolve method ambiguities. Hot-swapping the jar-file only might not work. + 4. ACLs should not be added to prefixed resources, (added in [KIP-290](https://cwiki.apache.org/confluence/display/KAFKA/KIP-290%3A+Support+for+Prefixed+ACLs)), until all brokers in the cluster have been updated. + +**NOTE:** any prefixed ACLs added to a cluster, even after the cluster is fully upgraded, will be ignored should the cluster be downgraded again. + + + +### Notable changes in 2.0.0 + + * [KIP-186](https://cwiki.apache.org/confluence/x/oYtjB) increases the default offset retention time from 1 day to 7 days. This makes it less likely to "lose" offsets in an application that commits infrequently. It also increases the active set of offsets and therefore can increase memory usage on the broker. Note that the console consumer currently enables offset commit by default and can be the source of a large number of offsets which this change will now preserve for 7 days instead of 1. You can preserve the existing behavior by setting the broker config `offsets.retention.minutes` to 1440. + * Support for Java 7 has been dropped, Java 8 is now the minimum version required. + * The default value for `ssl.endpoint.identification.algorithm` was changed to `https`, which performs hostname verification (man-in-the-middle attacks are possible otherwise). Set `ssl.endpoint.identification.algorithm` to an empty string to restore the previous behaviour. + * [KAFKA-5674](https://issues.apache.org/jira/browse/KAFKA-5674) extends the lower interval of `max.connections.per.ip` minimum to zero and therefore allows IP-based filtering of inbound connections. + * [KIP-272](https://cwiki.apache.org/confluence/display/KAFKA/KIP-272%3A+Add+API+version+tag+to+broker%27s+RequestsPerSec+metric) added API version tag to the metric `kafka.network:type=RequestMetrics,name=RequestsPerSec,request={Produce|FetchConsumer|FetchFollower|...}`. This metric now becomes `kafka.network:type=RequestMetrics,name=RequestsPerSec,request={Produce|FetchConsumer|FetchFollower|...},version={0|1|2|3|...}`. This will impact JMX monitoring tools that do not automatically aggregate. To get the total count for a specific request type, the tool needs to be updated to aggregate across different versions. + * [KIP-225](https://cwiki.apache.org/confluence/x/uaBzB) changed the metric "records.lag" to use tags for topic and partition. The original version with the name format "{topic}-{partition}.records-lag" has been removed. + * The Scala consumers, which have been deprecated since 0.11.0.0, have been removed. The Java consumer has been the recommended option since 0.10.0.0. Note that the Scala consumers in 1.1.0 (and older) will continue to work even if the brokers are upgraded to 2.0.0. + * The Scala producers, which have been deprecated since 0.10.0.0, have been removed. The Java producer has been the recommended option since 0.9.0.0. Note that the behaviour of the default partitioner in the Java producer differs from the default partitioner in the Scala producers. Users migrating should consider configuring a custom partitioner that retains the previous behaviour. Note that the Scala producers in 1.1.0 (and older) will continue to work even if the brokers are upgraded to 2.0.0. + * MirrorMaker and ConsoleConsumer no longer support the Scala consumer, they always use the Java consumer. + * The ConsoleProducer no longer supports the Scala producer, it always uses the Java producer. + * A number of deprecated tools that rely on the Scala clients have been removed: ReplayLogProducer, SimpleConsumerPerformance, SimpleConsumerShell, ExportZkOffsets, ImportZkOffsets, UpdateOffsetsInZK, VerifyConsumerRebalance. + * The deprecated kafka.tools.ProducerPerformance has been removed, please use org.apache.kafka.tools.ProducerPerformance. + * New Kafka Streams configuration parameter `upgrade.from` added that allows rolling bounce upgrade from older version. + * [KIP-284](https://cwiki.apache.org/confluence/x/DVyHB) changed the retention time for Kafka Streams repartition topics by setting its default value to `Long.MAX_VALUE`. + * Updated `ProcessorStateManager` APIs in Kafka Streams for registering state stores to the processor topology. For more details please read the Streams [Upgrade Guide](/37/streams/upgrade-guide#streams_api_changes_200). + * In earlier releases, Connect's worker configuration required the `internal.key.converter` and `internal.value.converter` properties. In 2.0, these are [no longer required](https://cwiki.apache.org/confluence/x/AZQ7B) and default to the JSON converter. You may safely remove these properties from your Connect standalone and distributed worker configurations: +`internal.key.converter=org.apache.kafka.connect.json.JsonConverter` `internal.key.converter.schemas.enable=false` `internal.value.converter=org.apache.kafka.connect.json.JsonConverter` `internal.value.converter.schemas.enable=false` + * [KIP-266](https://cwiki.apache.org/confluence/x/5kiHB) adds a new consumer configuration `default.api.timeout.ms` to specify the default timeout to use for `KafkaConsumer` APIs that could block. The KIP also adds overloads for such blocking APIs to support specifying a specific timeout to use for each of them instead of using the default timeout set by `default.api.timeout.ms`. In particular, a new `poll(Duration)` API has been added which does not block for dynamic partition assignment. The old `poll(long)` API has been deprecated and will be removed in a future version. Overloads have also been added for other `KafkaConsumer` methods like `partitionsFor`, `listTopics`, `offsetsForTimes`, `beginningOffsets`, `endOffsets` and `close` that take in a `Duration`. + * Also as part of KIP-266, the default value of `request.timeout.ms` has been changed to 30 seconds. The previous value was a little higher than 5 minutes to account for maximum time that a rebalance would take. Now we treat the JoinGroup request in the rebalance as a special case and use a value derived from `max.poll.interval.ms` for the request timeout. All other request types use the timeout defined by `request.timeout.ms` + * The internal method `kafka.admin.AdminClient.deleteRecordsBefore` has been removed. Users are encouraged to migrate to `org.apache.kafka.clients.admin.AdminClient.deleteRecords`. + * The AclCommand tool `--producer` convenience option uses the [KIP-277](https://cwiki.apache.org/confluence/display/KAFKA/KIP-277+-+Fine+Grained+ACL+for+CreateTopics+API) finer grained ACL on the given topic. + * [KIP-176](https://cwiki.apache.org/confluence/display/KAFKA/KIP-176%3A+Remove+deprecated+new-consumer+option+for+tools) removes the `--new-consumer` option for all consumer based tools. This option is redundant since the new consumer is automatically used if --bootstrap-server is defined. + * [KIP-290](https://cwiki.apache.org/confluence/display/KAFKA/KIP-290%3A+Support+for+Prefixed+ACLs) adds the ability to define ACLs on prefixed resources, e.g. any topic starting with 'foo'. + * [KIP-283](https://cwiki.apache.org/confluence/display/KAFKA/KIP-283%3A+Efficient+Memory+Usage+for+Down-Conversion) improves message down-conversion handling on Kafka broker, which has typically been a memory-intensive operation. The KIP adds a mechanism by which the operation becomes less memory intensive by down-converting chunks of partition data at a time which helps put an upper bound on memory consumption. With this improvement, there is a change in `FetchResponse` protocol behavior where the broker could send an oversized message batch towards the end of the response with an invalid offset. Such oversized messages must be ignored by consumer clients, as is done by `KafkaConsumer`. + +KIP-283 also adds new topic and broker configurations `message.downconversion.enable` and `log.message.downconversion.enable` respectively to control whether down-conversion is enabled. When disabled, broker does not perform any down-conversion and instead sends an `UNSUPPORTED_VERSION` error to the client. + + * Dynamic broker configuration options can be stored in ZooKeeper using kafka-configs.sh before brokers are started. This option can be used to avoid storing clear passwords in server.properties as all password configs may be stored encrypted in ZooKeeper. + * ZooKeeper hosts are now re-resolved if connection attempt fails. But if your ZooKeeper host names resolve to multiple addresses and some of them are not reachable, then you may need to increase the connection timeout `zookeeper.connection.timeout.ms`. + + + +### New Protocol Versions + + * [KIP-279](https://cwiki.apache.org/confluence/display/KAFKA/KIP-279%3A+Fix+log+divergence+between+leader+and+follower+after+fast+leader+fail+over): OffsetsForLeaderEpochResponse v1 introduces a partition-level `leader_epoch` field. + * [KIP-219](https://cwiki.apache.org/confluence/display/KAFKA/KIP-219+-+Improve+quota+communication): Bump up the protocol versions of non-cluster action requests and responses that are throttled on quota violation. + * [KIP-290](https://cwiki.apache.org/confluence/display/KAFKA/KIP-290%3A+Support+for+Prefixed+ACLs): Bump up the protocol versions ACL create, describe and delete requests and responses. + + + +### Upgrading a 1.1 Kafka Streams Application + + * Upgrading your Streams application from 1.1 to 2.0 does not require a broker upgrade. A Kafka Streams 2.0 application can connect to 2.0, 1.1, 1.0, 0.11.0, 0.10.2 and 0.10.1 brokers (it is not possible to connect to 0.10.0 brokers though). + * Note that in 2.0 we have removed the public APIs that are deprecated prior to 1.0; users leveraging on those deprecated APIs need to make code changes accordingly. See [Streams API changes in 2.0.0](/37/streams/upgrade-guide#streams_api_changes_200) for more details. + + + +## Upgrading from 0.8.x, 0.9.x, 0.10.0.x, 0.10.1.x, 0.10.2.x, 0.11.0.x, or 1.0.x to 1.1.x + +Kafka 1.1.0 introduces wire protocol changes. By following the recommended rolling upgrade plan below, you guarantee no downtime during the upgrade. However, please review the notable changes in 1.1.0 before upgrading. + +**For a rolling upgrade:** + + 1. Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.8.2, 0.9.0, 0.10.0, 0.10.1, 0.10.2, 0.11.0, 1.0). + * log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact following the upgrade for the details on what this configuration does.) +If you are upgrading from 0.11.0.x or 1.0.x and you have not overridden the message format, then you only need to override the inter-broker protocol format. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (0.11.0 or 1.0). + 2. Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. + 3. Once the entire cluster is upgraded, bump the protocol version by editing `inter.broker.protocol.version` and setting it to 1.1. + 4. Restart the brokers one by one for the new protocol version to take effect. + 5. If you have overridden the message format version as instructed above, then you need to do one more rolling restart to upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later, change log.message.format.version to 1.1 on each broker and restart them one by one. Note that the older Scala consumer does not support the new message format introduced in 0.11, so to avoid the performance cost of down-conversion (or to take advantage of exactly once semantics), the newer Java consumer must be used. + + + +**Additional Upgrade Notes:** + + 1. If you are willing to accept downtime, you can simply take all the brokers down, update the code and start them back up. They will start with the new protocol by default. + 2. Bumping the protocol version and restarting can be done any time after the brokers are upgraded. It does not have to be immediately after. Similarly for the message format version. + 3. If you are using Java8 method references in your Kafka Streams code you might need to update your code to resolve method ambiguties. Hot-swapping the jar-file only might not work. + + + +### Notable changes in 1.1.1 + + * New Kafka Streams configuration parameter `upgrade.from` added that allows rolling bounce upgrade from version 0.10.0.x + * See the [**Kafka Streams upgrade guide**](/37/streams/upgrade-guide.html) for details about this new config. + + +### Notable changes in 1.1.0 + + * The kafka artifact in Maven no longer depends on log4j or slf4j-log4j12. Similarly to the kafka-clients artifact, users can now choose the logging back-end by including the appropriate slf4j module (slf4j-log4j12, logback, etc.). The release tarball still includes log4j and slf4j-log4j12. + * [KIP-225](https://cwiki.apache.org/confluence/x/uaBzB) changed the metric "records.lag" to use tags for topic and partition. The original version with the name format "{topic}-{partition}.records-lag" is deprecated and will be removed in 2.0.0. + * Kafka Streams is more robust against broker communication errors. Instead of stopping the Kafka Streams client with a fatal exception, Kafka Streams tries to self-heal and reconnect to the cluster. Using the new `AdminClient` you have better control of how often Kafka Streams retries and can [configure](/37/streams/developer-guide/config-streams) fine-grained timeouts (instead of hard coded retries as in older version). + * Kafka Streams rebalance time was reduced further making Kafka Streams more responsive. + * Kafka Connect now supports message headers in both sink and source connectors, and to manipulate them via simple message transforms. Connectors must be changed to explicitly use them. A new `HeaderConverter` is introduced to control how headers are (de)serialized, and the new "SimpleHeaderConverter" is used by default to use string representations of values. + * kafka.tools.DumpLogSegments now automatically sets deep-iteration option if print-data-log is enabled explicitly or implicitly due to any of the other options like decoder. + + + +### New Protocol Versions + + * [KIP-226](https://cwiki.apache.org/confluence/display/KAFKA/KIP-226+-+Dynamic+Broker+Configuration) introduced DescribeConfigs Request/Response v1. + * [KIP-227](https://cwiki.apache.org/confluence/display/KAFKA/KIP-227%3A+Introduce+Incremental+FetchRequests+to+Increase+Partition+Scalability) introduced Fetch Request/Response v7. + + + +### Upgrading a 1.0 Kafka Streams Application + + * Upgrading your Streams application from 1.0 to 1.1 does not require a broker upgrade. A Kafka Streams 1.1 application can connect to 1.0, 0.11.0, 0.10.2 and 0.10.1 brokers (it is not possible to connect to 0.10.0 brokers though). + * See [Streams API changes in 1.1.0](/37/streams/upgrade-guide#streams_api_changes_110) for more details. + + + +## Upgrading from 0.8.x, 0.9.x, 0.10.0.x, 0.10.1.x, 0.10.2.x or 0.11.0.x to 1.0.0 + +Kafka 1.0.0 introduces wire protocol changes. By following the recommended rolling upgrade plan below, you guarantee no downtime during the upgrade. However, please review the notable changes in 1.0.0 before upgrading. + +**For a rolling upgrade:** + + 1. Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.8.2, 0.9.0, 0.10.0, 0.10.1, 0.10.2, 0.11.0). + * log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact following the upgrade for the details on what this configuration does.) +If you are upgrading from 0.11.0.x and you have not overridden the message format, you must set both the message format version and the inter-broker protocol version to 0.11.0. + * inter.broker.protocol.version=0.11.0 + * log.message.format.version=0.11.0 + 2. Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. + 3. Once the entire cluster is upgraded, bump the protocol version by editing `inter.broker.protocol.version` and setting it to 1.0. + 4. Restart the brokers one by one for the new protocol version to take effect. + 5. If you have overridden the message format version as instructed above, then you need to do one more rolling restart to upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later, change log.message.format.version to 1.0 on each broker and restart them one by one. If you are upgrading from 0.11.0 and log.message.format.version is set to 0.11.0, you can update the config and skip the rolling restart. Note that the older Scala consumer does not support the new message format introduced in 0.11, so to avoid the performance cost of down-conversion (or to take advantage of exactly once semantics), the newer Java consumer must be used. + + + +**Additional Upgrade Notes:** + + 1. If you are willing to accept downtime, you can simply take all the brokers down, update the code and start them back up. They will start with the new protocol by default. + 2. Bumping the protocol version and restarting can be done any time after the brokers are upgraded. It does not have to be immediately after. Similarly for the message format version. + + + +### Notable changes in 1.0.2 + + * New Kafka Streams configuration parameter `upgrade.from` added that allows rolling bounce upgrade from version 0.10.0.x + * See the [**Kafka Streams upgrade guide**](/37/streams/upgrade-guide.html) for details about this new config. + + +### Notable changes in 1.0.1 + + * Restored binary compatibility of AdminClient's Options classes (e.g. CreateTopicsOptions, DeleteTopicsOptions, etc.) with 0.11.0.x. Binary (but not source) compatibility had been broken inadvertently in 1.0.0. + + + +### Notable changes in 1.0.0 + + * Topic deletion is now enabled by default, since the functionality is now stable. Users who wish to to retain the previous behavior should set the broker config `delete.topic.enable` to `false`. Keep in mind that topic deletion removes data and the operation is not reversible (i.e. there is no "undelete" operation) + * For topics that support timestamp search if no offset can be found for a partition, that partition is now included in the search result with a null offset value. Previously, the partition was not included in the map. This change was made to make the search behavior consistent with the case of topics not supporting timestamp search. + * If the `inter.broker.protocol.version` is 1.0 or later, a broker will now stay online to serve replicas on live log directories even if there are offline log directories. A log directory may become offline due to IOException caused by hardware failure. Users need to monitor the per-broker metric `offlineLogDirectoryCount` to check whether there is offline log directory. + * Added KafkaStorageException which is a retriable exception. KafkaStorageException will be converted to NotLeaderForPartitionException in the response if the version of the client's FetchRequest or ProducerRequest does not support KafkaStorageException. + * -XX:+DisableExplicitGC was replaced by -XX:+ExplicitGCInvokesConcurrent in the default JVM settings. This helps avoid out of memory exceptions during allocation of native memory by direct buffers in some cases. + * The overridden `handleError` method implementations have been removed from the following deprecated classes in the `kafka.api` package: `FetchRequest`, `GroupCoordinatorRequest`, `OffsetCommitRequest`, `OffsetFetchRequest`, `OffsetRequest`, `ProducerRequest`, and `TopicMetadataRequest`. This was only intended for use on the broker, but it is no longer in use and the implementations have not been maintained. A stub implementation has been retained for binary compatibility. + * The Java clients and tools now accept any string as a client-id. + * The deprecated tool `kafka-consumer-offset-checker.sh` has been removed. Use `kafka-consumer-groups.sh` to get consumer group details. + * SimpleAclAuthorizer now logs access denials to the authorizer log by default. + * Authentication failures are now reported to clients as one of the subclasses of `AuthenticationException`. No retries will be performed if a client connection fails authentication. + * Custom `SaslServer` implementations may throw `SaslAuthenticationException` to provide an error message to return to clients indicating the reason for authentication failure. Implementors should take care not to include any security-critical information in the exception message that should not be leaked to unauthenticated clients. + * The `app-info` mbean registered with JMX to provide version and commit id will be deprecated and replaced with metrics providing these attributes. + * Kafka metrics may now contain non-numeric values. `org.apache.kafka.common.Metric#value()` has been deprecated and will return `0.0` in such cases to minimise the probability of breaking users who read the value of every client metric (via a `MetricsReporter` implementation or by calling the `metrics()` method). `org.apache.kafka.common.Metric#metricValue()` can be used to retrieve numeric and non-numeric metric values. + * Every Kafka rate metric now has a corresponding cumulative count metric with the suffix `-total` to simplify downstream processing. For example, `records-consumed-rate` has a corresponding metric named `records-consumed-total`. + * Mx4j will only be enabled if the system property `kafka_mx4jenable` is set to `true`. Due to a logic inversion bug, it was previously enabled by default and disabled if `kafka_mx4jenable` was set to `true`. + * The package `org.apache.kafka.common.security.auth` in the clients jar has been made public and added to the javadocs. Internal classes which had previously been located in this package have been moved elsewhere. + * When using an Authorizer and a user doesn't have required permissions on a topic, the broker will return TOPIC_AUTHORIZATION_FAILED errors to requests irrespective of topic existence on broker. If the user have required permissions and the topic doesn't exists, then the UNKNOWN_TOPIC_OR_PARTITION error code will be returned. + * config/consumer.properties file updated to use new consumer config properties. + + + +### New Protocol Versions + + * [KIP-112](https://cwiki.apache.org/confluence/display/KAFKA/KIP-112%3A+Handle+disk+failure+for+JBOD): LeaderAndIsrRequest v1 introduces a partition-level `is_new` field. + * [KIP-112](https://cwiki.apache.org/confluence/display/KAFKA/KIP-112%3A+Handle+disk+failure+for+JBOD): UpdateMetadataRequest v4 introduces a partition-level `offline_replicas` field. + * [KIP-112](https://cwiki.apache.org/confluence/display/KAFKA/KIP-112%3A+Handle+disk+failure+for+JBOD): MetadataResponse v5 introduces a partition-level `offline_replicas` field. + * [KIP-112](https://cwiki.apache.org/confluence/display/KAFKA/KIP-112%3A+Handle+disk+failure+for+JBOD): ProduceResponse v4 introduces error code for KafkaStorageException. + * [KIP-112](https://cwiki.apache.org/confluence/display/KAFKA/KIP-112%3A+Handle+disk+failure+for+JBOD): FetchResponse v6 introduces error code for KafkaStorageException. + * [KIP-152](https://cwiki.apache.org/confluence/display/KAFKA/KIP-152+-+Improve+diagnostics+for+SASL+authentication+failures): SaslAuthenticate request has been added to enable reporting of authentication failures. This request will be used if the SaslHandshake request version is greater than 0. + + + +### Upgrading a 0.11.0 Kafka Streams Application + + * Upgrading your Streams application from 0.11.0 to 1.0 does not require a broker upgrade. A Kafka Streams 1.0 application can connect to 0.11.0, 0.10.2 and 0.10.1 brokers (it is not possible to connect to 0.10.0 brokers though). However, Kafka Streams 1.0 requires 0.10 message format or newer and does not work with older message formats. + * If you are monitoring on streams metrics, you will need make some changes to the metrics names in your reporting and monitoring code, because the metrics sensor hierarchy was changed. + * There are a few public APIs including `ProcessorContext#schedule()`, `Processor#punctuate()` and `KStreamBuilder`, `TopologyBuilder` are being deprecated by new APIs. We recommend making corresponding code changes, which should be very minor since the new APIs look quite similar, when you upgrade. + * See [Streams API changes in 1.0.0](/37/streams/upgrade-guide#streams_api_changes_100) for more details. + + + +### Upgrading a 0.10.2 Kafka Streams Application + + * Upgrading your Streams application from 0.10.2 to 1.0 does not require a broker upgrade. A Kafka Streams 1.0 application can connect to 1.0, 0.11.0, 0.10.2 and 0.10.1 brokers (it is not possible to connect to 0.10.0 brokers though). + * If you are monitoring on streams metrics, you will need make some changes to the metrics names in your reporting and monitoring code, because the metrics sensor hierarchy was changed. + * There are a few public APIs including `ProcessorContext#schedule()`, `Processor#punctuate()` and `KStreamBuilder`, `TopologyBuilder` are being deprecated by new APIs. We recommend making corresponding code changes, which should be very minor since the new APIs look quite similar, when you upgrade. + * If you specify customized `key.serde`, `value.serde` and `timestamp.extractor` in configs, it is recommended to use their replaced configure parameter as these configs are deprecated. + * See [Streams API changes in 0.11.0](/37/streams/upgrade-guide#streams_api_changes_0110) for more details. + + + +### Upgrading a 0.10.1 Kafka Streams Application + + * Upgrading your Streams application from 0.10.1 to 1.0 does not require a broker upgrade. A Kafka Streams 1.0 application can connect to 1.0, 0.11.0, 0.10.2 and 0.10.1 brokers (it is not possible to connect to 0.10.0 brokers though). + * You need to recompile your code. Just swapping the Kafka Streams library jar file will not work and will break your application. + * If you are monitoring on streams metrics, you will need make some changes to the metrics names in your reporting and monitoring code, because the metrics sensor hierarchy was changed. + * There are a few public APIs including `ProcessorContext#schedule()`, `Processor#punctuate()` and `KStreamBuilder`, `TopologyBuilder` are being deprecated by new APIs. We recommend making corresponding code changes, which should be very minor since the new APIs look quite similar, when you upgrade. + * If you specify customized `key.serde`, `value.serde` and `timestamp.extractor` in configs, it is recommended to use their replaced configure parameter as these configs are deprecated. + * If you use a custom (i.e., user implemented) timestamp extractor, you will need to update this code, because the `TimestampExtractor` interface was changed. + * If you register custom metrics, you will need to update this code, because the `StreamsMetric` interface was changed. + * See [Streams API changes in 1.0.0](/37/streams/upgrade-guide#streams_api_changes_100), [Streams API changes in 0.11.0](/37/streams/upgrade-guide#streams_api_changes_0110) and [Streams API changes in 0.10.2](/37/streams/upgrade-guide#streams_api_changes_0102) for more details. + + + +### Upgrading a 0.10.0 Kafka Streams Application + + * Upgrading your Streams application from 0.10.0 to 1.0 does require a broker upgrade because a Kafka Streams 1.0 application can only connect to 0.1, 0.11.0, 0.10.2, or 0.10.1 brokers. + * There are couple of API changes, that are not backward compatible (cf. [Streams API changes in 1.0.0](/37/streams/upgrade-guide#streams_api_changes_100), [Streams API changes in 0.11.0](/37/streams#streams_api_changes_0110), [Streams API changes in 0.10.2](/37/streams#streams_api_changes_0102), and [Streams API changes in 0.10.1](/37/streams#streams_api_changes_0101) for more details). Thus, you need to update and recompile your code. Just swapping the Kafka Streams library jar file will not work and will break your application. + * Upgrading from 0.10.0.x to 1.0.2 requires two rolling bounces with config `upgrade.from="0.10.0"` set for first upgrade phase (cf. [KIP-268](https://cwiki.apache.org/confluence/display/KAFKA/KIP-268%3A+Simplify+Kafka+Streams+Rebalance+Metadata+Upgrade)). As an alternative, an offline upgrade is also possible. + * prepare your application instances for a rolling bounce and make sure that config `upgrade.from` is set to `"0.10.0"` for new version 0.11.0.3 + * bounce each instance of your application once + * prepare your newly deployed 1.0.2 application instances for a second round of rolling bounces; make sure to remove the value for config `upgrade.from` + * bounce each instance of your application once more to complete the upgrade + * Upgrading from 0.10.0.x to 1.0.0 or 1.0.1 requires an offline upgrade (rolling bounce upgrade is not supported) + * stop all old (0.10.0.x) application instances + * update your code and swap old code and jar file with new code and new jar file + * restart all new (1.0.0 or 1.0.1) application instances + + + +## Upgrading from 0.8.x, 0.9.x, 0.10.0.x, 0.10.1.x or 0.10.2.x to 0.11.0.0 + +Kafka 0.11.0.0 introduces a new message format version as well as wire protocol changes. By following the recommended rolling upgrade plan below, you guarantee no downtime during the upgrade. However, please review the notable changes in 0.11.0.0 before upgrading. + +Starting with version 0.10.2, Java clients (producer and consumer) have acquired the ability to communicate with older brokers. Version 0.11.0 clients can talk to version 0.10.0 or newer brokers. However, if your brokers are older than 0.10.0, you must upgrade all the brokers in the Kafka cluster before upgrading your clients. Version 0.11.0 brokers support 0.8.x and newer clients. + +**For a rolling upgrade:** + + 1. Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the current message format version currently in use. If you have not overridden the message format previously, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION. + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.8.2, 0.9.0, 0.10.0, 0.10.1 or 0.10.2). + * log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact following the upgrade for the details on what this configuration does.) + 2. Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. + 3. Once the entire cluster is upgraded, bump the protocol version by editing `inter.broker.protocol.version` and setting it to 0.11.0, but do not change `log.message.format.version` yet. + 4. Restart the brokers one by one for the new protocol version to take effect. + 5. Once all (or most) consumers have been upgraded to 0.11.0 or later, then change log.message.format.version to 0.11.0 on each broker and restart them one by one. Note that the older Scala consumer does not support the new message format, so to avoid the performance cost of down-conversion (or to take advantage of exactly once semantics), the new Java consumer must be used. + + + +**Additional Upgrade Notes:** + + 1. If you are willing to accept downtime, you can simply take all the brokers down, update the code and start them back up. They will start with the new protocol by default. + 2. Bumping the protocol version and restarting can be done any time after the brokers are upgraded. It does not have to be immediately after. Similarly for the message format version. + 3. It is also possible to enable the 0.11.0 message format on individual topics using the topic admin tool (`bin/kafka-topics.sh`) prior to updating the global setting `log.message.format.version`. + 4. If you are upgrading from a version prior to 0.10.0, it is NOT necessary to first update the message format to 0.10.0 before you switch to 0.11.0. + + + +### Upgrading a 0.10.2 Kafka Streams Application + + * Upgrading your Streams application from 0.10.2 to 0.11.0 does not require a broker upgrade. A Kafka Streams 0.11.0 application can connect to 0.11.0, 0.10.2 and 0.10.1 brokers (it is not possible to connect to 0.10.0 brokers though). + * If you specify customized `key.serde`, `value.serde` and `timestamp.extractor` in configs, it is recommended to use their replaced configure parameter as these configs are deprecated. + * See [Streams API changes in 0.11.0](/37/streams/upgrade-guide#streams_api_changes_0110) for more details. + + + +### Upgrading a 0.10.1 Kafka Streams Application + + * Upgrading your Streams application from 0.10.1 to 0.11.0 does not require a broker upgrade. A Kafka Streams 0.11.0 application can connect to 0.11.0, 0.10.2 and 0.10.1 brokers (it is not possible to connect to 0.10.0 brokers though). + * You need to recompile your code. Just swapping the Kafka Streams library jar file will not work and will break your application. + * If you specify customized `key.serde`, `value.serde` and `timestamp.extractor` in configs, it is recommended to use their replaced configure parameter as these configs are deprecated. + * If you use a custom (i.e., user implemented) timestamp extractor, you will need to update this code, because the `TimestampExtractor` interface was changed. + * If you register custom metrics, you will need to update this code, because the `StreamsMetric` interface was changed. + * See [Streams API changes in 0.11.0](/37/streams/upgrade-guide#streams_api_changes_0110) and [Streams API changes in 0.10.2](/37/streams/upgrade-guide#streams_api_changes_0102) for more details. + + + +### Upgrading a 0.10.0 Kafka Streams Application + + * Upgrading your Streams application from 0.10.0 to 0.11.0 does require a broker upgrade because a Kafka Streams 0.11.0 application can only connect to 0.11.0, 0.10.2, or 0.10.1 brokers. + * There are couple of API changes, that are not backward compatible (cf. [Streams API changes in 0.11.0](/37/streams#streams_api_changes_0110), [Streams API changes in 0.10.2](/37/streams#streams_api_changes_0102), and [Streams API changes in 0.10.1](/37/streams#streams_api_changes_0101) for more details). Thus, you need to update and recompile your code. Just swapping the Kafka Streams library jar file will not work and will break your application. + * Upgrading from 0.10.0.x to 0.11.0.3 requires two rolling bounces with config `upgrade.from="0.10.0"` set for first upgrade phase (cf. [KIP-268](https://cwiki.apache.org/confluence/display/KAFKA/KIP-268%3A+Simplify+Kafka+Streams+Rebalance+Metadata+Upgrade)). As an alternative, an offline upgrade is also possible. + * prepare your application instances for a rolling bounce and make sure that config `upgrade.from` is set to `"0.10.0"` for new version 0.11.0.3 + * bounce each instance of your application once + * prepare your newly deployed 0.11.0.3 application instances for a second round of rolling bounces; make sure to remove the value for config `upgrade.from` + * bounce each instance of your application once more to complete the upgrade + * Upgrading from 0.10.0.x to 0.11.0.0, 0.11.0.1, or 0.11.0.2 requires an offline upgrade (rolling bounce upgrade is not supported) + * stop all old (0.10.0.x) application instances + * update your code and swap old code and jar file with new code and new jar file + * restart all new (0.11.0.0 , 0.11.0.1, or 0.11.0.2) application instances + + + +### Notable changes in 0.11.0.3 + + * New Kafka Streams configuration parameter `upgrade.from` added that allows rolling bounce upgrade from version 0.10.0.x + * See the [**Kafka Streams upgrade guide**](/37/streams/upgrade-guide.html) for details about this new config. + + +### Notable changes in 0.11.0.0 + + * Unclean leader election is now disabled by default. The new default favors durability over availability. Users who wish to to retain the previous behavior should set the broker config `unclean.leader.election.enable` to `true`. + * Producer configs `block.on.buffer.full`, `metadata.fetch.timeout.ms` and `timeout.ms` have been removed. They were initially deprecated in Kafka 0.9.0.0. + * The `offsets.topic.replication.factor` broker config is now enforced upon auto topic creation. Internal auto topic creation will fail with a GROUP_COORDINATOR_NOT_AVAILABLE error until the cluster size meets this replication factor requirement. + * When compressing data with snappy, the producer and broker will use the compression scheme's default block size (2 x 32 KB) instead of 1 KB in order to improve the compression ratio. There have been reports of data compressed with the smaller block size being 50% larger than when compressed with the larger block size. For the snappy case, a producer with 5000 partitions will require an additional 315 MB of JVM heap. + * Similarly, when compressing data with gzip, the producer and broker will use 8 KB instead of 1 KB as the buffer size. The default for gzip is excessively low (512 bytes). + * The broker configuration `max.message.bytes` now applies to the total size of a batch of messages. Previously the setting applied to batches of compressed messages, or to non-compressed messages individually. A message batch may consist of only a single message, so in most cases, the limitation on the size of individual messages is only reduced by the overhead of the batch format. However, there are some subtle implications for message format conversion (see below for more detail). Note also that while previously the broker would ensure that at least one message is returned in each fetch request (regardless of the total and partition-level fetch sizes), the same behavior now applies to one message batch. + * GC log rotation is enabled by default, see KAFKA-3754 for details. + * Deprecated constructors of RecordMetadata, MetricName and Cluster classes have been removed. + * Added user headers support through a new Headers interface providing user headers read and write access. + * ProducerRecord and ConsumerRecord expose the new Headers API via `Headers headers()` method call. + * ExtendedSerializer and ExtendedDeserializer interfaces are introduced to support serialization and deserialization for headers. Headers will be ignored if the configured serializer and deserializer are not the above classes. + * A new config, `group.initial.rebalance.delay.ms`, was introduced. This config specifies the time, in milliseconds, that the `GroupCoordinator` will delay the initial consumer rebalance. The rebalance will be further delayed by the value of `group.initial.rebalance.delay.ms` as new members join the group, up to a maximum of `max.poll.interval.ms`. The default value for this is 3 seconds. During development and testing it might be desirable to set this to 0 in order to not delay test execution time. + * `org.apache.kafka.common.Cluster#partitionsForTopic`, `partitionsForNode` and `availablePartitionsForTopic` methods will return an empty list instead of `null` (which is considered a bad practice) in case the metadata for the required topic does not exist. + * Streams API configuration parameters `timestamp.extractor`, `key.serde`, and `value.serde` were deprecated and replaced by `default.timestamp.extractor`, `default.key.serde`, and `default.value.serde`, respectively. + * For offset commit failures in the Java consumer's `commitAsync` APIs, we no longer expose the underlying cause when instances of `RetriableCommitFailedException` are passed to the commit callback. See [KAFKA-5052](https://issues.apache.org/jira/browse/KAFKA-5052) for more detail. + + + +### New Protocol Versions + + * [KIP-107](https://cwiki.apache.org/confluence/display/KAFKA/KIP-107%3A+Add+purgeDataBefore\(\)+API+in+AdminClient): FetchRequest v5 introduces a partition-level `log_start_offset` field. + * [KIP-107](https://cwiki.apache.org/confluence/display/KAFKA/KIP-107%3A+Add+purgeDataBefore\(\)+API+in+AdminClient): FetchResponse v5 introduces a partition-level `log_start_offset` field. + * [KIP-82](https://cwiki.apache.org/confluence/display/KAFKA/KIP-82+-+Add+Record+Headers): ProduceRequest v3 introduces an array of `header` in the message protocol, containing `key` field and `value` field. + * [KIP-82](https://cwiki.apache.org/confluence/display/KAFKA/KIP-82+-+Add+Record+Headers): FetchResponse v5 introduces an array of `header` in the message protocol, containing `key` field and `value` field. + + + +### Notes on Exactly Once Semantics + +Kafka 0.11.0 includes support for idempotent and transactional capabilities in the producer. Idempotent delivery ensures that messages are delivered exactly once to a particular topic partition during the lifetime of a single producer. Transactional delivery allows producers to send data to multiple partitions such that either all messages are successfully delivered, or none of them are. Together, these capabilities enable "exactly once semantics" in Kafka. More details on these features are available in the user guide, but below we add a few specific notes on enabling them in an upgraded cluster. Note that enabling EoS is not required and there is no impact on the broker's behavior if unused. + + 1. Only the new Java producer and consumer support exactly once semantics. + 2. These features depend crucially on the 0.11.0 message format. Attempting to use them on an older format will result in unsupported version errors. + 3. Transaction state is stored in a new internal topic `__transaction_state`. This topic is not created until the the first attempt to use a transactional request API. Similar to the consumer offsets topic, there are several settings to control the topic's configuration. For example, `transaction.state.log.min.isr` controls the minimum ISR for this topic. See the configuration section in the user guide for a full list of options. + 4. For secure clusters, the transactional APIs require new ACLs which can be turned on with the `bin/kafka-acls.sh`. tool. + 5. EoS in Kafka introduces new request APIs and modifies several existing ones. See [KIP-98](https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging#KIP-98-ExactlyOnceDeliveryandTransactionalMessaging-RPCProtocolSummary) for the full details + + + +### Notes on the new message format in 0.11.0 + +The 0.11.0 message format includes several major enhancements in order to support better delivery semantics for the producer (see [KIP-98](https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging)) and improved replication fault tolerance (see [KIP-101](https://cwiki.apache.org/confluence/display/KAFKA/KIP-101+-+Alter+Replication+Protocol+to+use+Leader+Epoch+rather+than+High+Watermark+for+Truncation)). Although the new format contains more information to make these improvements possible, we have made the batch format much more efficient. As long as the number of messages per batch is more than 2, you can expect lower overall overhead. For smaller batches, however, there may be a small performance impact. See [here](bit.ly/kafka-eos-perf) for the results of our initial performance analysis of the new message format. You can also find more detail on the message format in the [KIP-98](https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging#KIP-98-ExactlyOnceDeliveryandTransactionalMessaging-MessageFormat) proposal. + +One of the notable differences in the new message format is that even uncompressed messages are stored together as a single batch. This has a few implications for the broker configuration `max.message.bytes`, which limits the size of a single batch. First, if an older client produces messages to a topic partition using the old format, and the messages are individually smaller than `max.message.bytes`, the broker may still reject them after they are merged into a single batch during the up-conversion process. Generally this can happen when the aggregate size of the individual messages is larger than `max.message.bytes`. There is a similar effect for older consumers reading messages down-converted from the new format: if the fetch size is not set at least as large as `max.message.bytes`, the consumer may not be able to make progress even if the individual uncompressed messages are smaller than the configured fetch size. This behavior does not impact the Java client for 0.10.1.0 and later since it uses an updated fetch protocol which ensures that at least one message can be returned even if it exceeds the fetch size. To get around these problems, you should ensure 1) that the producer's batch size is not set larger than `max.message.bytes`, and 2) that the consumer's fetch size is set at least as large as `max.message.bytes`. + +Most of the discussion on the performance impact of upgrading to the 0.10.0 message format remains pertinent to the 0.11.0 upgrade. This mainly affects clusters that are not secured with TLS since "zero-copy" transfer is already not possible in that case. In order to avoid the cost of down-conversion, you should ensure that consumer applications are upgraded to the latest 0.11.0 client. Significantly, since the old consumer has been deprecated in 0.11.0.0, it does not support the new message format. You must upgrade to use the new consumer to use the new message format without the cost of down-conversion. Note that 0.11.0 consumers support backwards compatibility with 0.10.0 brokers and upward, so it is possible to upgrade the clients first before the brokers. + +## Upgrading from 0.8.x, 0.9.x, 0.10.0.x or 0.10.1.x to 0.10.2.0 + +0.10.2.0 has wire protocol changes. By following the recommended rolling upgrade plan below, you guarantee no downtime during the upgrade. However, please review the notable changes in 0.10.2.0 before upgrading. + +Starting with version 0.10.2, Java clients (producer and consumer) have acquired the ability to communicate with older brokers. Version 0.10.2 clients can talk to version 0.10.0 or newer brokers. However, if your brokers are older than 0.10.0, you must upgrade all the brokers in the Kafka cluster before upgrading your clients. Version 0.10.2 brokers support 0.8.x and newer clients. + +**For a rolling upgrade:** + + 1. Update server.properties file on all brokers and add the following properties: + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.8.2, 0.9.0, 0.10.0 or 0.10.1). + * log.message.format.version=CURRENT_KAFKA_VERSION (See potential performance impact following the upgrade for the details on what this configuration does.) + 2. Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. + 3. Once the entire cluster is upgraded, bump the protocol version by editing inter.broker.protocol.version and setting it to 0.10.2. + 4. If your previous message format is 0.10.0, change log.message.format.version to 0.10.2 (this is a no-op as the message format is the same for 0.10.0, 0.10.1 and 0.10.2). If your previous message format version is lower than 0.10.0, do not change log.message.format.version yet - this parameter should only change once all consumers have been upgraded to 0.10.0.0 or later. + 5. Restart the brokers one by one for the new protocol version to take effect. + 6. If log.message.format.version is still lower than 0.10.0 at this point, wait until all consumers have been upgraded to 0.10.0 or later, then change log.message.format.version to 0.10.2 on each broker and restart them one by one. + + + +**Note:** If you are willing to accept downtime, you can simply take all the brokers down, update the code and start all of them. They will start with the new protocol by default. + +**Note:** Bumping the protocol version and restarting can be done any time after the brokers were upgraded. It does not have to be immediately after. + +### Upgrading a 0.10.1 Kafka Streams Application + + * Upgrading your Streams application from 0.10.1 to 0.10.2 does not require a broker upgrade. A Kafka Streams 0.10.2 application can connect to 0.10.2 and 0.10.1 brokers (it is not possible to connect to 0.10.0 brokers though). + * You need to recompile your code. Just swapping the Kafka Streams library jar file will not work and will break your application. + * If you use a custom (i.e., user implemented) timestamp extractor, you will need to update this code, because the `TimestampExtractor` interface was changed. + * If you register custom metrics, you will need to update this code, because the `StreamsMetric` interface was changed. + * See [Streams API changes in 0.10.2](/37/streams/upgrade-guide#streams_api_changes_0102) for more details. + + + +### Upgrading a 0.10.0 Kafka Streams Application + + * Upgrading your Streams application from 0.10.0 to 0.10.2 does require a broker upgrade because a Kafka Streams 0.10.2 application can only connect to 0.10.2 or 0.10.1 brokers. + * There are couple of API changes, that are not backward compatible (cf. [Streams API changes in 0.10.2](/37/streams#streams_api_changes_0102) for more details). Thus, you need to update and recompile your code. Just swapping the Kafka Streams library jar file will not work and will break your application. + * Upgrading from 0.10.0.x to 0.10.2.2 requires two rolling bounces with config `upgrade.from="0.10.0"` set for first upgrade phase (cf. [KIP-268](https://cwiki.apache.org/confluence/display/KAFKA/KIP-268%3A+Simplify+Kafka+Streams+Rebalance+Metadata+Upgrade)). As an alternative, an offline upgrade is also possible. + * prepare your application instances for a rolling bounce and make sure that config `upgrade.from` is set to `"0.10.0"` for new version 0.10.2.2 + * bounce each instance of your application once + * prepare your newly deployed 0.10.2.2 application instances for a second round of rolling bounces; make sure to remove the value for config `upgrade.from` + * bounce each instance of your application once more to complete the upgrade + * Upgrading from 0.10.0.x to 0.10.2.0 or 0.10.2.1 requires an offline upgrade (rolling bounce upgrade is not supported) + * stop all old (0.10.0.x) application instances + * update your code and swap old code and jar file with new code and new jar file + * restart all new (0.10.2.0 or 0.10.2.1) application instances + + + +### Notable changes in 0.10.2.2 + + * New configuration parameter `upgrade.from` added that allows rolling bounce upgrade from version 0.10.0.x + + + +### Notable changes in 0.10.2.1 + + * The default values for two configurations of the StreamsConfig class were changed to improve the resiliency of Kafka Streams applications. The internal Kafka Streams producer `retries` default value was changed from 0 to 10. The internal Kafka Streams consumer `max.poll.interval.ms` default value was changed from 300000 to `Integer.MAX_VALUE`. + + + +### Notable changes in 0.10.2.0 + + * The Java clients (producer and consumer) have acquired the ability to communicate with older brokers. Version 0.10.2 clients can talk to version 0.10.0 or newer brokers. Note that some features are not available or are limited when older brokers are used. + * Several methods on the Java consumer may now throw `InterruptException` if the calling thread is interrupted. Please refer to the `KafkaConsumer` Javadoc for a more in-depth explanation of this change. + * Java consumer now shuts down gracefully. By default, the consumer waits up to 30 seconds to complete pending requests. A new close API with timeout has been added to `KafkaConsumer` to control the maximum wait time. + * Multiple regular expressions separated by commas can be passed to MirrorMaker with the new Java consumer via the --whitelist option. This makes the behaviour consistent with MirrorMaker when used the old Scala consumer. + * Upgrading your Streams application from 0.10.1 to 0.10.2 does not require a broker upgrade. A Kafka Streams 0.10.2 application can connect to 0.10.2 and 0.10.1 brokers (it is not possible to connect to 0.10.0 brokers though). + * The Zookeeper dependency was removed from the Streams API. The Streams API now uses the Kafka protocol to manage internal topics instead of modifying Zookeeper directly. This eliminates the need for privileges to access Zookeeper directly and "StreamsConfig.ZOOKEEPER_CONFIG" should not be set in the Streams app any more. If the Kafka cluster is secured, Streams apps must have the required security privileges to create new topics. + * Several new fields including "security.protocol", "connections.max.idle.ms", "retry.backoff.ms", "reconnect.backoff.ms" and "request.timeout.ms" were added to StreamsConfig class. User should pay attention to the default values and set these if needed. For more details please refer to [3.5 Kafka Streams Configs](/37/#streamsconfigs). + + + +### New Protocol Versions + + * [KIP-88](https://cwiki.apache.org/confluence/display/KAFKA/KIP-88%3A+OffsetFetch+Protocol+Update): OffsetFetchRequest v2 supports retrieval of offsets for all topics if the `topics` array is set to `null`. + * [KIP-88](https://cwiki.apache.org/confluence/display/KAFKA/KIP-88%3A+OffsetFetch+Protocol+Update): OffsetFetchResponse v2 introduces a top-level `error_code` field. + * [KIP-103](https://cwiki.apache.org/confluence/display/KAFKA/KIP-103%3A+Separation+of+Internal+and+External+traffic): UpdateMetadataRequest v3 introduces a `listener_name` field to the elements of the `end_points` array. + * [KIP-108](https://cwiki.apache.org/confluence/display/KAFKA/KIP-108%3A+Create+Topic+Policy): CreateTopicsRequest v1 introduces a `validate_only` field. + * [KIP-108](https://cwiki.apache.org/confluence/display/KAFKA/KIP-108%3A+Create+Topic+Policy): CreateTopicsResponse v1 introduces an `error_message` field to the elements of the `topic_errors` array. + + + +## Upgrading from 0.8.x, 0.9.x or 0.10.0.X to 0.10.1.0 + +0.10.1.0 has wire protocol changes. By following the recommended rolling upgrade plan below, you guarantee no downtime during the upgrade. However, please notice the Potential breaking changes in 0.10.1.0 before upgrade. +Note: Because new protocols are introduced, it is important to upgrade your Kafka clusters before upgrading your clients (i.e. 0.10.1.x clients only support 0.10.1.x or later brokers while 0.10.1.x brokers also support older clients). + +**For a rolling upgrade:** + + 1. Update server.properties file on all brokers and add the following properties: + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.8.2.0, 0.9.0.0 or 0.10.0.0). + * log.message.format.version=CURRENT_KAFKA_VERSION (See potential performance impact following the upgrade for the details on what this configuration does.) + 2. Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. + 3. Once the entire cluster is upgraded, bump the protocol version by editing inter.broker.protocol.version and setting it to 0.10.1.0. + 4. If your previous message format is 0.10.0, change log.message.format.version to 0.10.1 (this is a no-op as the message format is the same for both 0.10.0 and 0.10.1). If your previous message format version is lower than 0.10.0, do not change log.message.format.version yet - this parameter should only change once all consumers have been upgraded to 0.10.0.0 or later. + 5. Restart the brokers one by one for the new protocol version to take effect. + 6. If log.message.format.version is still lower than 0.10.0 at this point, wait until all consumers have been upgraded to 0.10.0 or later, then change log.message.format.version to 0.10.1 on each broker and restart them one by one. + + + +**Note:** If you are willing to accept downtime, you can simply take all the brokers down, update the code and start all of them. They will start with the new protocol by default. + +**Note:** Bumping the protocol version and restarting can be done any time after the brokers were upgraded. It does not have to be immediately after. + +### Notable changes in 0.10.1.2 + + * New configuration parameter `upgrade.from` added that allows rolling bounce upgrade from version 0.10.0.x + + + +### Potential breaking changes in 0.10.1.0 + + * The log retention time is no longer based on last modified time of the log segments. Instead it will be based on the largest timestamp of the messages in a log segment. + * The log rolling time is no longer depending on log segment create time. Instead it is now based on the timestamp in the messages. More specifically. if the timestamp of the first message in the segment is T, the log will be rolled out when a new message has a timestamp greater than or equal to T + log.roll.ms + * The open file handlers of 0.10.0 will increase by ~33% because of the addition of time index files for each segment. + * The time index and offset index share the same index size configuration. Since each time index entry is 1.5x the size of offset index entry. User may need to increase log.index.size.max.bytes to avoid potential frequent log rolling. + * Due to the increased number of index files, on some brokers with large amount the log segments (e.g. >15K), the log loading process during the broker startup could be longer. Based on our experiment, setting the num.recovery.threads.per.data.dir to one may reduce the log loading time. + + + +### Upgrading a 0.10.0 Kafka Streams Application + + * Upgrading your Streams application from 0.10.0 to 0.10.1 does require a broker upgrade because a Kafka Streams 0.10.1 application can only connect to 0.10.1 brokers. + * There are couple of API changes, that are not backward compatible (cf. [Streams API changes in 0.10.1](/37/streams/upgrade-guide#streams_api_changes_0101) for more details). Thus, you need to update and recompile your code. Just swapping the Kafka Streams library jar file will not work and will break your application. + * Upgrading from 0.10.0.x to 0.10.1.2 requires two rolling bounces with config `upgrade.from="0.10.0"` set for first upgrade phase (cf. [KIP-268](https://cwiki.apache.org/confluence/display/KAFKA/KIP-268%3A+Simplify+Kafka+Streams+Rebalance+Metadata+Upgrade)). As an alternative, an offline upgrade is also possible. + * prepare your application instances for a rolling bounce and make sure that config `upgrade.from` is set to `"0.10.0"` for new version 0.10.1.2 + * bounce each instance of your application once + * prepare your newly deployed 0.10.1.2 application instances for a second round of rolling bounces; make sure to remove the value for config `upgrade.from` + * bounce each instance of your application once more to complete the upgrade + * Upgrading from 0.10.0.x to 0.10.1.0 or 0.10.1.1 requires an offline upgrade (rolling bounce upgrade is not supported) + * stop all old (0.10.0.x) application instances + * update your code and swap old code and jar file with new code and new jar file + * restart all new (0.10.1.0 or 0.10.1.1) application instances + + + +### Notable changes in 0.10.1.0 + + * The new Java consumer is no longer in beta and we recommend it for all new development. The old Scala consumers are still supported, but they will be deprecated in the next release and will be removed in a future major release. + * The `--new-consumer`/`--new.consumer` switch is no longer required to use tools like MirrorMaker and the Console Consumer with the new consumer; one simply needs to pass a Kafka broker to connect to instead of the ZooKeeper ensemble. In addition, usage of the Console Consumer with the old consumer has been deprecated and it will be removed in a future major release. + * Kafka clusters can now be uniquely identified by a cluster id. It will be automatically generated when a broker is upgraded to 0.10.1.0. The cluster id is available via the kafka.server:type=KafkaServer,name=ClusterId metric and it is part of the Metadata response. Serializers, client interceptors and metric reporters can receive the cluster id by implementing the ClusterResourceListener interface. + * The BrokerState "RunningAsController" (value 4) has been removed. Due to a bug, a broker would only be in this state briefly before transitioning out of it and hence the impact of the removal should be minimal. The recommended way to detect if a given broker is the controller is via the kafka.controller:type=KafkaController,name=ActiveControllerCount metric. + * The new Java Consumer now allows users to search offsets by timestamp on partitions. + * The new Java Consumer now supports heartbeating from a background thread. There is a new configuration `max.poll.interval.ms` which controls the maximum time between poll invocations before the consumer will proactively leave the group (5 minutes by default). The value of the configuration `request.timeout.ms` (default to 30 seconds) must always be smaller than `max.poll.interval.ms`(default to 5 minutes), since that is the maximum time that a JoinGroup request can block on the server while the consumer is rebalance. Finally, the default value of `session.timeout.ms` has been adjusted down to 10 seconds, and the default value of `max.poll.records` has been changed to 500. + * When using an Authorizer and a user doesn't have **Describe** authorization on a topic, the broker will no longer return TOPIC_AUTHORIZATION_FAILED errors to requests since this leaks topic names. Instead, the UNKNOWN_TOPIC_OR_PARTITION error code will be returned. This may cause unexpected timeouts or delays when using the producer and consumer since Kafka clients will typically retry automatically on unknown topic errors. You should consult the client logs if you suspect this could be happening. + * Fetch responses have a size limit by default (50 MB for consumers and 10 MB for replication). The existing per partition limits also apply (1 MB for consumers and replication). Note that neither of these limits is an absolute maximum as explained in the next point. + * Consumers and replicas can make progress if a message larger than the response/partition size limit is found. More concretely, if the first message in the first non-empty partition of the fetch is larger than either or both limits, the message will still be returned. + * Overloaded constructors were added to `kafka.api.FetchRequest` and `kafka.javaapi.FetchRequest` to allow the caller to specify the order of the partitions (since order is significant in v3). The previously existing constructors were deprecated and the partitions are shuffled before the request is sent to avoid starvation issues. + + + +### New Protocol Versions + + * ListOffsetRequest v1 supports accurate offset search based on timestamps. + * MetadataResponse v2 introduces a new field: "cluster_id". + * FetchRequest v3 supports limiting the response size (in addition to the existing per partition limit), it returns messages bigger than the limits if required to make progress and the order of partitions in the request is now significant. + * JoinGroup v1 introduces a new field: "rebalance_timeout". + + + +## Upgrading from 0.8.x or 0.9.x to 0.10.0.0 + +0.10.0.0 has potential breaking changes (please review before upgrading) and possible performance impact following the upgrade. By following the recommended rolling upgrade plan below, you guarantee no downtime and no performance impact during and following the upgrade. +Note: Because new protocols are introduced, it is important to upgrade your Kafka clusters before upgrading your clients. + +**Notes to clients with version 0.9.0.0:** Due to a bug introduced in 0.9.0.0, clients that depend on ZooKeeper (old Scala high-level Consumer and MirrorMaker if used with the old consumer) will not work with 0.10.0.x brokers. Therefore, 0.9.0.0 clients should be upgraded to 0.9.0.1 **before** brokers are upgraded to 0.10.0.x. This step is not necessary for 0.8.X or 0.9.0.1 clients. + +**For a rolling upgrade:** + + 1. Update server.properties file on all brokers and add the following properties: + * inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.8.2 or 0.9.0.0). + * log.message.format.version=CURRENT_KAFKA_VERSION (See potential performance impact following the upgrade for the details on what this configuration does.) + 2. Upgrade the brokers. This can be done a broker at a time by simply bringing it down, updating the code, and restarting it. + 3. Once the entire cluster is upgraded, bump the protocol version by editing inter.broker.protocol.version and setting it to 0.10.0.0. NOTE: You shouldn't touch log.message.format.version yet - this parameter should only change once all consumers have been upgraded to 0.10.0.0 + 4. Restart the brokers one by one for the new protocol version to take effect. + 5. Once all consumers have been upgraded to 0.10.0, change log.message.format.version to 0.10.0 on each broker and restart them one by one. + + + +**Note:** If you are willing to accept downtime, you can simply take all the brokers down, update the code and start all of them. They will start with the new protocol by default. + +**Note:** Bumping the protocol version and restarting can be done any time after the brokers were upgraded. It does not have to be immediately after. + +### Potential performance impact following upgrade to 0.10.0.0 + +The message format in 0.10.0 includes a new timestamp field and uses relative offsets for compressed messages. The on disk message format can be configured through log.message.format.version in the server.properties file. The default on-disk message format is 0.10.0. If a consumer client is on a version before 0.10.0.0, it only understands message formats before 0.10.0. In this case, the broker is able to convert messages from the 0.10.0 format to an earlier format before sending the response to the consumer on an older version. However, the broker can't use zero-copy transfer in this case. Reports from the Kafka community on the performance impact have shown CPU utilization going from 20% before to 100% after an upgrade, which forced an immediate upgrade of all clients to bring performance back to normal. To avoid such message conversion before consumers are upgraded to 0.10.0.0, one can set log.message.format.version to 0.8.2 or 0.9.0 when upgrading the broker to 0.10.0.0. This way, the broker can still use zero-copy transfer to send the data to the old consumers. Once consumers are upgraded, one can change the message format to 0.10.0 on the broker and enjoy the new message format that includes new timestamp and improved compression. The conversion is supported to ensure compatibility and can be useful to support a few apps that have not updated to newer clients yet, but is impractical to support all consumer traffic on even an overprovisioned cluster. Therefore, it is critical to avoid the message conversion as much as possible when brokers have been upgraded but the majority of clients have not. + +For clients that are upgraded to 0.10.0.0, there is no performance impact. + +**Note:** By setting the message format version, one certifies that all existing messages are on or below that message format version. Otherwise consumers before 0.10.0.0 might break. In particular, after the message format is set to 0.10.0, one should not change it back to an earlier format as it may break consumers on versions before 0.10.0.0. + +**Note:** Due to the additional timestamp introduced in each message, producers sending small messages may see a message throughput degradation because of the increased overhead. Likewise, replication now transmits an additional 8 bytes per message. If you're running close to the network capacity of your cluster, it's possible that you'll overwhelm the network cards and see failures and performance issues due to the overload. + +**Note:** If you have enabled compression on producers, you may notice reduced producer throughput and/or lower compression rate on the broker in some cases. When receiving compressed messages, 0.10.0 brokers avoid recompressing the messages, which in general reduces the latency and improves the throughput. In certain cases, however, this may reduce the batching size on the producer, which could lead to worse throughput. If this happens, users can tune linger.ms and batch.size of the producer for better throughput. In addition, the producer buffer used for compressing messages with snappy is smaller than the one used by the broker, which may have a negative impact on the compression ratio for the messages on disk. We intend to make this configurable in a future Kafka release. + +### Potential breaking changes in 0.10.0.0 + + * Starting from Kafka 0.10.0.0, the message format version in Kafka is represented as the Kafka version. For example, message format 0.9.0 refers to the highest message version supported by Kafka 0.9.0. + * Message format 0.10.0 has been introduced and it is used by default. It includes a timestamp field in the messages and relative offsets are used for compressed messages. + * ProduceRequest/Response v2 has been introduced and it is used by default to support message format 0.10.0 + * FetchRequest/Response v2 has been introduced and it is used by default to support message format 0.10.0 + * MessageFormatter interface was changed from `def writeTo(key: Array[Byte], value: Array[Byte], output: PrintStream)` to `def writeTo(consumerRecord: ConsumerRecord[Array[Byte], Array[Byte]], output: PrintStream)` + * MessageReader interface was changed from `def readMessage(): KeyedMessage[Array[Byte], Array[Byte]]` to `def readMessage(): ProducerRecord[Array[Byte], Array[Byte]]` + * MessageFormatter's package was changed from `kafka.tools` to `kafka.common` + * MessageReader's package was changed from `kafka.tools` to `kafka.common` + * MirrorMakerMessageHandler no longer exposes the `handle(record: MessageAndMetadata[Array[Byte], Array[Byte]])` method as it was never called. + * The 0.7 KafkaMigrationTool is no longer packaged with Kafka. If you need to migrate from 0.7 to 0.10.0, please migrate to 0.8 first and then follow the documented upgrade process to upgrade from 0.8 to 0.10.0. + * The new consumer has standardized its APIs to accept `java.util.Collection` as the sequence type for method parameters. Existing code may have to be updated to work with the 0.10.0 client library. + * LZ4-compressed message handling was changed to use an interoperable framing specification (LZ4f v1.5.1). To maintain compatibility with old clients, this change only applies to Message format 0.10.0 and later. Clients that Produce/Fetch LZ4-compressed messages using v0/v1 (Message format 0.9.0) should continue to use the 0.9.0 framing implementation. Clients that use Produce/Fetch protocols v2 or later should use interoperable LZ4f framing. A list of interoperable LZ4 libraries is available at + + +### Notable changes in 0.10.0.0 + + * Starting from Kafka 0.10.0.0, a new client library named **Kafka Streams** is available for stream processing on data stored in Kafka topics. This new client library only works with 0.10.x and upward versioned brokers due to message format changes mentioned above. For more information please read [Streams documentation](/37/streams). + * The default value of the configuration parameter `receive.buffer.bytes` is now 64K for the new consumer. + * The new consumer now exposes the configuration parameter `exclude.internal.topics` to restrict internal topics (such as the consumer offsets topic) from accidentally being included in regular expression subscriptions. By default, it is enabled. + * The old Scala producer has been deprecated. Users should migrate their code to the Java producer included in the kafka-clients JAR as soon as possible. + * The new consumer API has been marked stable. + + + +## Upgrading from 0.8.0, 0.8.1.X, or 0.8.2.X to 0.9.0.0 + +0.9.0.0 has potential breaking changes (please review before upgrading) and an inter-broker protocol change from previous versions. This means that upgraded brokers and clients may not be compatible with older versions. It is important that you upgrade your Kafka cluster before upgrading your clients. If you are using MirrorMaker downstream clusters should be upgraded first as well. + +**For a rolling upgrade:** + + 1. Update server.properties file on all brokers and add the following property: inter.broker.protocol.version=0.8.2.X + 2. Upgrade the brokers. This can be done a broker at a time by simply bringing it down, updating the code, and restarting it. + 3. Once the entire cluster is upgraded, bump the protocol version by editing inter.broker.protocol.version and setting it to 0.9.0.0. + 4. Restart the brokers one by one for the new protocol version to take effect + + + +**Note:** If you are willing to accept downtime, you can simply take all the brokers down, update the code and start all of them. They will start with the new protocol by default. + +**Note:** Bumping the protocol version and restarting can be done any time after the brokers were upgraded. It does not have to be immediately after. + +### Potential breaking changes in 0.9.0.0 + + * Java 1.6 is no longer supported. + * Scala 2.9 is no longer supported. + * Broker IDs above 1000 are now reserved by default to automatically assigned broker IDs. If your cluster has existing broker IDs above that threshold make sure to increase the reserved.broker.max.id broker configuration property accordingly. + * Configuration parameter replica.lag.max.messages was removed. Partition leaders will no longer consider the number of lagging messages when deciding which replicas are in sync. + * Configuration parameter replica.lag.time.max.ms now refers not just to the time passed since last fetch request from replica, but also to time since the replica last caught up. Replicas that are still fetching messages from leaders but did not catch up to the latest messages in replica.lag.time.max.ms will be considered out of sync. + * Compacted topics no longer accept messages without key and an exception is thrown by the producer if this is attempted. In 0.8.x, a message without key would cause the log compaction thread to subsequently complain and quit (and stop compacting all compacted topics). + * MirrorMaker no longer supports multiple target clusters. As a result it will only accept a single --consumer.config parameter. To mirror multiple source clusters, you will need at least one MirrorMaker instance per source cluster, each with its own consumer configuration. + * Tools packaged under _org.apache.kafka.clients.tools.*_ have been moved to _org.apache.kafka.tools.*_. All included scripts will still function as usual, only custom code directly importing these classes will be affected. + * The default Kafka JVM performance options (KAFKA_JVM_PERFORMANCE_OPTS) have been changed in kafka-run-class.sh. + * The kafka-topics.sh script (kafka.admin.TopicCommand) now exits with non-zero exit code on failure. + * The kafka-topics.sh script (kafka.admin.TopicCommand) will now print a warning when topic names risk metric collisions due to the use of a '.' or '_' in the topic name, and error in the case of an actual collision. + * The kafka-console-producer.sh script (kafka.tools.ConsoleProducer) will use the Java producer instead of the old Scala producer be default, and users have to specify 'old-producer' to use the old producer. + * By default, all command line tools will print all logging messages to stderr instead of stdout. + + + +### Notable changes in 0.9.0.1 + + * The new broker id generation feature can be disabled by setting broker.id.generation.enable to false. + * Configuration parameter log.cleaner.enable is now true by default. This means topics with a cleanup.policy=compact will now be compacted by default, and 128 MB of heap will be allocated to the cleaner process via log.cleaner.dedupe.buffer.size. You may want to review log.cleaner.dedupe.buffer.size and the other log.cleaner configuration values based on your usage of compacted topics. + * Default value of configuration parameter fetch.min.bytes for the new consumer is now 1 by default. + + + +### Deprecations in 0.9.0.0 + + * Altering topic configuration from the kafka-topics.sh script (kafka.admin.TopicCommand) has been deprecated. Going forward, please use the kafka-configs.sh script (kafka.admin.ConfigCommand) for this functionality. + * The kafka-consumer-offset-checker.sh (kafka.tools.ConsumerOffsetChecker) has been deprecated. Going forward, please use kafka-consumer-groups.sh (kafka.admin.ConsumerGroupCommand) for this functionality. + * The kafka.tools.ProducerPerformance class has been deprecated. Going forward, please use org.apache.kafka.tools.ProducerPerformance for this functionality (kafka-producer-perf-test.sh will also be changed to use the new class). + * The producer config block.on.buffer.full has been deprecated and will be removed in future release. Currently its default value has been changed to false. The KafkaProducer will no longer throw BufferExhaustedException but instead will use max.block.ms value to block, after which it will throw a TimeoutException. If block.on.buffer.full property is set to true explicitly, it will set the max.block.ms to Long.MAX_VALUE and metadata.fetch.timeout.ms will not be honoured + + + +## Upgrading from 0.8.1 to 0.8.2 + +0.8.2 is fully compatible with 0.8.1. The upgrade can be done one broker at a time by simply bringing it down, updating the code, and restarting it. + +## Upgrading from 0.8.0 to 0.8.1 + +0.8.1 is fully compatible with 0.8. The upgrade can be done one broker at a time by simply bringing it down, updating the code, and restarting it. + +## Upgrading from 0.7 + +Release 0.7 is incompatible with newer releases. Major changes were made to the API, ZooKeeper data structures, and protocol, and configuration in order to add replication (Which was missing in 0.7). The upgrade from 0.7 to later versions requires a [special tool](https://cwiki.apache.org/confluence/display/KAFKA/Migrating+from+0.7+to+0.8) for migration. This migration can be done without downtime. diff --git a/docs/getting-started/uses.md b/docs/getting-started/uses.md new file mode 100644 index 0000000000000..e3715df790977 --- /dev/null +++ b/docs/getting-started/uses.md @@ -0,0 +1,45 @@ +--- +title: Use Cases +description: +weight: 2 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +Here is a description of a few of the popular use cases for Apache Kafka®. For an overview of a number of these areas in action, see [this blog post](https://engineering.linkedin.com/distributed-systems/log-what-every-software-engineer-should-know-about-real-time-datas-unifying/). + +## Messaging + +Kafka works well as a replacement for a more traditional message broker. Message brokers are used for a variety of reasons (to decouple processing from data producers, to buffer unprocessed messages, etc). In comparison to most messaging systems Kafka has better throughput, built-in partitioning, replication, and fault-tolerance which makes it a good solution for large scale message processing applications. + +In our experience messaging uses are often comparatively low-throughput, but may require low end-to-end latency and often depend on the strong durability guarantees Kafka provides. + +In this domain Kafka is comparable to traditional messaging systems such as [ActiveMQ](https://activemq.apache.org) or [RabbitMQ](https://www.rabbitmq.com). + +## Website Activity Tracking + +The original use case for Kafka was to be able to rebuild a user activity tracking pipeline as a set of real-time publish-subscribe feeds. This means site activity (page views, searches, or other actions users may take) is published to central topics with one topic per activity type. These feeds are available for subscription for a range of use cases including real-time processing, real-time monitoring, and loading into Hadoop or offline data warehousing systems for offline processing and reporting. + +Activity tracking is often very high volume as many activity messages are generated for each user page view. + +## Metrics + +Kafka is often used for operational monitoring data. This involves aggregating statistics from distributed applications to produce centralized feeds of operational data. + +## Log Aggregation + +Many people use Kafka as a replacement for a log aggregation solution. Log aggregation typically collects physical log files off servers and puts them in a central place (a file server or HDFS perhaps) for processing. Kafka abstracts away the details of files and gives a cleaner abstraction of log or event data as a stream of messages. This allows for lower-latency processing and easier support for multiple data sources and distributed data consumption. In comparison to log-centric systems like Scribe or Flume, Kafka offers equally good performance, stronger durability guarantees due to replication, and much lower end-to-end latency. + +## Stream Processing + +Many users of Kafka process data in processing pipelines consisting of multiple stages, where raw input data is consumed from Kafka topics and then aggregated, enriched, or otherwise transformed into new topics for further consumption or follow-up processing. For example, a processing pipeline for recommending news articles might crawl article content from RSS feeds and publish it to an "articles" topic; further processing might normalize or deduplicate this content and publish the cleansed article content to a new topic; a final processing stage might attempt to recommend this content to users. Such processing pipelines create graphs of real-time data flows based on the individual topics. Starting in 0.10.0.0, a light-weight but powerful stream processing library called [Kafka Streams](/streams) is available in Apache Kafka to perform such data processing as described above. Apart from Kafka Streams, alternative open source stream processing tools include [Apache Storm](https://storm.apache.org/) and [Apache Samza](https://samza.apache.org/). + +## Event Sourcing + +[Event sourcing](https://martinfowler.com/eaaDev/EventSourcing.html) is a style of application design where state changes are logged as a time-ordered sequence of records. Kafka's support for very large stored log data makes it an excellent backend for an application built in this style. + +## Commit Log + +Kafka can serve as a kind of external commit-log for a distributed system. The log helps replicate data between nodes and acts as a re-syncing mechanism for failed nodes to restore their data. The [log compaction](/documentation.html#compaction) feature in Kafka helps support this usage. In this usage Kafka is similar to [Apache BookKeeper](https://bookkeeper.apache.org/) project. diff --git a/docs/implementation.html b/docs/implementation.html deleted file mode 100644 index fb1f52ade5bcd..0000000000000 --- a/docs/implementation.html +++ /dev/null @@ -1,297 +0,0 @@ - - - - -
diff --git a/docs/implementation/_index.md b/docs/implementation/_index.md new file mode 100644 index 0000000000000..7426a2273f789 --- /dev/null +++ b/docs/implementation/_index.md @@ -0,0 +1,10 @@ +--- +title: Implementation +description: +weight: 5 +tags: ['kafka', 'docs', 'implementation'] +aliases: +keywords: +type: docs +--- + diff --git a/docs/implementation/distribution.md b/docs/implementation/distribution.md new file mode 100644 index 0000000000000..0d8fcb09ed20f --- /dev/null +++ b/docs/implementation/distribution.md @@ -0,0 +1,51 @@ +--- +title: Distribution +description: Distribution +weight: 5 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +## Consumer Offset Tracking + +Kafka consumer tracks the maximum offset it has consumed in each partition and has the capability to commit offsets so that it can resume from those offsets in the event of a restart. Kafka provides the option to store all the offsets for a given consumer group in a designated broker (for that group) called the group coordinator. i.e., any consumer instance in that consumer group should send its offset commits and fetches to that group coordinator (broker). Consumer groups are assigned to coordinators based on their group names. A consumer can look up its coordinator by issuing a FindCoordinatorRequest to any Kafka broker and reading the FindCoordinatorResponse which will contain the coordinator details. The consumer can then proceed to commit or fetch offsets from the coordinator broker. In case the coordinator moves, the consumer will need to rediscover the coordinator. Offset commits can be done automatically or manually by consumer instance. + +When the group coordinator receives an OffsetCommitRequest, it appends the request to a special compacted Kafka topic named ___consumer_offsets_. The broker sends a successful offset commit response to the consumer only after all the replicas of the offsets topic receive the offsets. In case the offsets fail to replicate within a configurable timeout, the offset commit will fail and the consumer may retry the commit after backing off. The brokers periodically compact the offsets topic since it only needs to maintain the most recent offset commit per partition. The coordinator also caches the offsets in an in-memory table in order to serve offset fetches quickly. + +When the coordinator receives an offset fetch request, it simply returns the last committed offset vector from the offsets cache. In case coordinator was just started or if it just became the coordinator for a new set of consumer groups (by becoming a leader for a partition of the offsets topic), it may need to load the offsets topic partition into the cache. In this case, the offset fetch will fail with an CoordinatorLoadInProgressException and the consumer may retry the OffsetFetchRequest after backing off. + +## ZooKeeper Directories + +The following gives the ZooKeeper structures and algorithms used for co-ordination between consumers and brokers. + +## Notation + +When an element in a path is denoted `[xyz]`, that means that the value of xyz is not fixed and there is in fact a ZooKeeper znode for each possible value of xyz. For example `/topics/[topic]` would be a directory named /topics containing a sub-directory for each topic name. Numerical ranges are also given such as `[0...5]` to indicate the subdirectories 0, 1, 2, 3, 4. An arrow `->` is used to indicate the contents of a znode. For example `/hello -> world` would indicate a znode /hello containing the value "world". + +## Broker Node Registry + + + /brokers/ids/[0...N] --> {"jmx_port":...,"timestamp":...,"endpoints":[...],"host":...,"version":...,"port":...} (ephemeral node) + +This is a list of all present broker nodes, each of which provides a unique logical broker id which identifies it to consumers (which must be given as part of its configuration). On startup, a broker node registers itself by creating a znode with the logical broker id under /brokers/ids. The purpose of the logical broker id is to allow a broker to be moved to a different physical machine without affecting consumers. An attempt to register a broker id that is already in use (say because two servers are configured with the same broker id) results in an error. + +Since the broker registers itself in ZooKeeper using ephemeral znodes, this registration is dynamic and will disappear if the broker is shutdown or dies (thus notifying consumers it is no longer available). + +## Broker Topic Registry + + + /brokers/topics/[topic]/partitions/[0...N]/state --> {"controller_epoch":...,"leader":...,"version":...,"leader_epoch":...,"isr":[...]} (ephemeral node) + +Each broker registers itself under the topics it maintains and stores the number of partitions for that topic. + +## Cluster Id + +The cluster id is a unique and immutable identifier assigned to a Kafka cluster. The cluster id can have a maximum of 22 characters and the allowed characters are defined by the regular expression [a-zA-Z0-9_\\-]+, which corresponds to the characters used by the URL-safe Base64 variant with no padding. Conceptually, it is auto-generated when a cluster is started for the first time. + +Implementation-wise, it is generated when a broker with version 0.10.1 or later is successfully started for the first time. The broker tries to get the cluster id from the `/cluster/id` znode during startup. If the znode does not exist, the broker generates a new cluster id and creates the znode with this cluster id. + +## Broker node registration + +The broker nodes are basically independent, so they only publish information about what they have. When a broker joins, it registers itself under the broker node registry directory and writes information about its host name and port. The broker also register the list of existing topics and their logical partitions in the broker topic registry. New topics are registered dynamically when they are created on the broker. diff --git a/docs/implementation/log.md b/docs/implementation/log.md new file mode 100644 index 0000000000000..8be1bfb1c5764 --- /dev/null +++ b/docs/implementation/log.md @@ -0,0 +1,59 @@ +--- +title: Log +description: Log +weight: 4 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +A log for a topic named "my-topic" with two partitions consists of two directories (namely `my-topic-0` and `my-topic-1`) populated with data files containing the messages for that topic. The format of the log files is a sequence of "log entries"; each log entry is a 4 byte integer _N_ storing the message length which is followed by the _N_ message bytes. Each message is uniquely identified by a 64-bit integer _offset_ giving the byte position of the start of this message in the stream of all messages ever sent to that topic on that partition. The on-disk format of each message is given below. Each log file is named with the offset of the first message it contains. So the first file created will be 00000000000000000000.log, and each additional file will have an integer name roughly _S_ bytes from the previous file where _S_ is the max log file size given in the configuration. + +The exact binary format for records is versioned and maintained as a standard interface so record batches can be transferred between producer, broker, and client without recopying or conversion when desirable. The previous section included details about the on-disk format of records. + +The use of the message offset as the message id is unusual. Our original idea was to use a GUID generated by the producer, and maintain a mapping from GUID to offset on each broker. But since a consumer must maintain an ID for each server, the global uniqueness of the GUID provides no value. Furthermore, the complexity of maintaining the mapping from a random id to an offset requires a heavy weight index structure which must be synchronized with disk, essentially requiring a full persistent random-access data structure. Thus to simplify the lookup structure we decided to use a simple per-partition atomic counter which could be coupled with the partition id and node id to uniquely identify a message; this makes the lookup structure simpler, though multiple seeks per consumer request are still likely. However once we settled on a counter, the jump to directly using the offset seemed natural--both after all are monotonically increasing integers unique to a partition. Since the offset is hidden from the consumer API this decision is ultimately an implementation detail and we went with the more efficient approach. + +![](/37/images/kafka_log.png) + +## Writes + +The log allows serial appends which always go to the last file. This file is rolled over to a fresh file when it reaches a configurable size (say 1GB). The log takes two configuration parameters: _M_ , which gives the number of messages to write before forcing the OS to flush the file to disk, and _S_ , which gives a number of seconds after which a flush is forced. This gives a durability guarantee of losing at most _M_ messages or _S_ seconds of data in the event of a system crash. + +## Reads + +Reads are done by giving the 64-bit logical offset of a message and an _S_ -byte max chunk size. This will return an iterator over the messages contained in the _S_ -byte buffer. _S_ is intended to be larger than any single message, but in the event of an abnormally large message, the read can be retried multiple times, each time doubling the buffer size, until the message is read successfully. A maximum message and buffer size can be specified to make the server reject messages larger than some size, and to give a bound to the client on the maximum it needs to ever read to get a complete message. It is likely that the read buffer ends with a partial message, this is easily detected by the size delimiting. + +The actual process of reading from an offset requires first locating the log segment file in which the data is stored, calculating the file-specific offset from the global offset value, and then reading from that file offset. The search is done as a simple binary search variation against an in-memory range maintained for each file. + +The log provides the capability of getting the most recently written message to allow clients to start subscribing as of "right now". This is also useful in the case the consumer fails to consume its data within its SLA-specified number of days. In this case when the client attempts to consume a non-existent offset it is given an OutOfRangeException and can either reset itself or fail as appropriate to the use case. + +The following is the format of the results sent to the consumer. + + + MessageSetSend (fetch result) + + total length : 4 bytes + error code : 2 bytes + message 1 : x bytes + ... + message n : x bytes + + + MultiMessageSetSend (multiFetch result) + + total length : 4 bytes + error code : 2 bytes + messageSetSend 1 + ... + messageSetSend n + +## Deletes + +Data is deleted one log segment at a time. The log manager applies two metrics to identify segments which are eligible for deletion: time and size. For time-based policies, the record timestamps are considered, with the largest timestamp in a segment file (order of records is not relevant) defining the retention time for the entire segment. Size-based retention is disabled by default. When enabled the log manager keeps deleting the oldest segment file until the overall size of the partition is within the configured limit again. If both policies are enabled at the same time, a segment that is eligible for deletion due to either policy will be deleted. To avoid locking reads while still allowing deletes that modify the segment list we use a copy-on-write style segment list implementation that provides consistent views to allow a binary search to proceed on an immutable static snapshot view of the log segments while deletes are progressing. + +## Guarantees + +The log provides a configuration parameter _M_ which controls the maximum number of messages that are written before forcing a flush to disk. On startup a log recovery process is run that iterates over all messages in the newest log segment and verifies that each message entry is valid. A message entry is valid if the sum of its size and offset are less than the length of the file AND the CRC32 of the message payload matches the CRC stored with the message. In the event corruption is detected the log is truncated to the last valid offset. + +Note that two kinds of corruption must be handled: truncation in which an unwritten block is lost due to a crash, and corruption in which a nonsense block is ADDED to the file. The reason for this is that in general the OS makes no guarantee of the write order between the file inode and the actual block data so in addition to losing written data the file can gain nonsense data if the inode is updated with a new size but a crash occurs before the block containing that data is written. The CRC detects this corner case, and prevents it from corrupting the log (though the unwritten messages are, of course, lost). diff --git a/docs/implementation/message-format.md b/docs/implementation/message-format.md new file mode 100644 index 0000000000000..68fa6ec0a2552 --- /dev/null +++ b/docs/implementation/message-format.md @@ -0,0 +1,145 @@ +--- +title: Message Format +description: Message Format +weight: 3 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +Messages (aka Records) are always written in batches. The technical term for a batch of messages is a record batch, and a record batch contains one or more records. In the degenerate case, we could have a record batch containing a single record. Record batches and records have their own headers. The format of each is described below. + +## Record Batch + +The following is the on-disk format of a RecordBatch. + + + baseOffset: int64 + batchLength: int32 + partitionLeaderEpoch: int32 + magic: int8 (current magic value is 2) + crc: uint32 + attributes: int16 + bit 0~2: + 0: no compression + 1: gzip + 2: snappy + 3: lz4 + 4: zstd + bit 3: timestampType + bit 4: isTransactional (0 means not transactional) + bit 5: isControlBatch (0 means not a control batch) + bit 6: hasDeleteHorizonMs (0 means baseTimestamp is not set as the delete horizon for compaction) + bit 7~15: unused + lastOffsetDelta: int32 + baseTimestamp: int64 + maxTimestamp: int64 + producerId: int64 + producerEpoch: int16 + baseSequence: int32 + records: [Record] + +Note that when compression is enabled, the compressed record data is serialized directly following the count of the number of records. + +The CRC covers the data from the attributes to the end of the batch (i.e. all the bytes that follow the CRC). It is located after the magic byte, which means that clients must parse the magic byte before deciding how to interpret the bytes between the batch length and the magic byte. The partition leader epoch field is not included in the CRC computation to avoid the need to recompute the CRC when this field is assigned for every batch that is received by the broker. The CRC-32C (Castagnoli) polynomial is used for the computation. + +On compaction: unlike the older message formats, magic v2 and above preserves the first and last offset/sequence numbers from the original batch when the log is cleaned. This is required in order to be able to restore the producer's state when the log is reloaded. If we did not retain the last sequence number, for example, then after a partition leader failure, the producer might see an OutOfSequence error. The base sequence number must be preserved for duplicate checking (the broker checks incoming Produce requests for duplicates by verifying that the first and last sequence numbers of the incoming batch match the last from that producer). As a result, it is possible to have empty batches in the log when all the records in the batch are cleaned but batch is still retained in order to preserve a producer's last sequence number. One oddity here is that the baseTimestamp field is not preserved during compaction, so it will change if the first record in the batch is compacted away. + +Compaction may also modify the baseTimestamp if the record batch contains records with a null payload or aborted transaction markers. The baseTimestamp will be set to the timestamp of when those records should be deleted with the delete horizon attribute bit also set. + +### Control Batches + +A control batch contains a single record called the control record. Control records should not be passed on to applications. Instead, they are used by consumers to filter out aborted transactional messages. + +The key of a control record conforms to the following schema: + + + version: int16 (current version is 0) + type: int16 (0 indicates an abort marker, 1 indicates a commit) + +The schema for the value of a control record is dependent on the type. The value is opaque to clients. + +## Record + +Record level headers were introduced in Kafka 0.11.0. The on-disk format of a record with Headers is delineated below. + + + length: varint + attributes: int8 + bit 0~7: unused + timestampDelta: varlong + offsetDelta: varint + keyLength: varint + key: byte[] + valueLen: varint + value: byte[] + Headers => [Header] + +### Record Header + + + headerKeyLength: varint + headerKey: String + headerValueLength: varint + Value: byte[] + +We use the same varint encoding as Protobuf. More information on the latter can be found [here](https://developers.google.com/protocol-buffers/docs/encoding#varints). The count of headers in a record is also encoded as a varint. + +## Old Message Format + +Prior to Kafka 0.11, messages were transferred and stored in _message sets_. In a message set, each message has its own metadata. Note that although message sets are represented as an array, they are not preceded by an int32 array size like other array elements in the protocol. + +**Message Set:** + + + + MessageSet (Version: 0) => [offset message_size message] + offset => INT64 + message_size => INT32 + message => crc magic_byte attributes key value + crc => INT32 + magic_byte => INT8 + attributes => INT8 + bit 0~2: + 0: no compression + 1: gzip + 2: snappy + bit 3~7: unused + key => BYTES + value => BYTES + + + MessageSet (Version: 1) => [offset message_size message] + offset => INT64 + message_size => INT32 + message => crc magic_byte attributes timestamp key value + crc => INT32 + magic_byte => INT8 + attributes => INT8 + bit 0~2: + 0: no compression + 1: gzip + 2: snappy + 3: lz4 + bit 3: timestampType + 0: create time + 1: log append time + bit 4~7: unused + timestamp => INT64 + key => BYTES + value => BYTES + +In versions prior to Kafka 0.10, the only supported message format version (which is indicated in the magic value) was 0. Message format version 1 was introduced with timestamp support in version 0.10. + + * Similarly to version 2 above, the lowest bits of attributes represent the compression type. + * In version 1, the producer should always set the timestamp type bit to 0. If the topic is configured to use log append time, (through either broker level config log.message.timestamp.type = LogAppendTime or topic level config message.timestamp.type = LogAppendTime), the broker will overwrite the timestamp type and the timestamp in the message set. + * The highest bits of attributes must be set to 0. + + + +In message format versions 0 and 1 Kafka supports recursive messages to enable compression. In this case the message's attributes must be set to indicate one of the compression types and the value field will contain a message set compressed with that type. We often refer to the nested messages as "inner messages" and the wrapping message as the "outer message." Note that the key should be null for the outer message and its offset will be the offset of the last inner message. + +When receiving recursive version 0 messages, the broker decompresses them and each inner message is assigned an offset individually. In version 1, to avoid server side re-compression, only the wrapper message will be assigned an offset. The inner messages will have relative offsets. The absolute offset can be computed using the offset from the outer message, which corresponds to the offset assigned to the last inner message. + +The crc field contains the CRC32 (and not CRC-32C) of the subsequent message bytes (i.e. from magic byte to the value). diff --git a/docs/implementation/messages.md b/docs/implementation/messages.md new file mode 100644 index 0000000000000..0f887c370a0e9 --- /dev/null +++ b/docs/implementation/messages.md @@ -0,0 +1,11 @@ +--- +title: Messages +description: Messages +weight: 2 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +Messages consist of a variable-length header, a variable-length opaque key byte array and a variable-length opaque value byte array. The format of the header is described in the following section. Leaving the key and value opaque is the right decision: there is a great deal of progress being made on serialization libraries right now, and any particular choice is unlikely to be right for all uses. Needless to say a particular application using Kafka would likely mandate a particular serialization type as part of its usage. The `RecordBatch` interface is simply an iterator over messages with specialized methods for bulk reading and writing to an NIO `Channel`. diff --git a/docs/implementation/network-layer.md b/docs/implementation/network-layer.md new file mode 100644 index 0000000000000..a49b3d7e2be79 --- /dev/null +++ b/docs/implementation/network-layer.md @@ -0,0 +1,11 @@ +--- +title: Network Layer +description: Network Layer +weight: 1 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +The network layer is a fairly straight-forward NIO server, and will not be described in great detail. The sendfile implementation is done by giving the `TransferableRecords` interface a `writeTo` method. This allows the file-backed message set to use the more efficient `transferTo` implementation instead of an in-process buffered write. The threading model is a single acceptor thread and _N_ processor threads which handle a fixed number of connections each. This design has been pretty thoroughly tested [elsewhere](https://web.archive.org/web/20120619234320/https://sna-projects.com/blog/2009/08/introducing-the-nio-socketserver-implementation/) and found to be simple to implement and fast. The protocol is kept quite simple to allow for future implementation of clients in other languages. diff --git a/docs/introduction.html b/docs/introduction.html deleted file mode 100644 index 49de2faf030dd..0000000000000 --- a/docs/introduction.html +++ /dev/null @@ -1,220 +0,0 @@ - - - - - - -
diff --git a/docs/js/templateData.js b/docs/js/templateData.js deleted file mode 100644 index 15537f680cd51..0000000000000 --- a/docs/js/templateData.js +++ /dev/null @@ -1,24 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one or more -contributor license agreements. See the NOTICE file distributed with -this work for additional information regarding copyright ownership. -The ASF licenses this file to You under the Apache License, Version 2.0 -(the "License"); you may not use this file except in compliance with -the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -// Define variables for doc templates -var context={ - "version": "37", - "dotVersion": "3.7", - "fullDotVersion": "3.7.3-SNAPSHOT", - "scalaVersion": "2.13" -}; diff --git a/docs/kafka-connect/_index.md b/docs/kafka-connect/_index.md new file mode 100644 index 0000000000000..9f00c35b328af --- /dev/null +++ b/docs/kafka-connect/_index.md @@ -0,0 +1,10 @@ +--- +title: Kafka Connect +description: +weight: 8 +tags: ['kafka', 'docs', 'security'] +aliases: +keywords: +type: docs +--- + diff --git a/docs/kafka-connect/administration.md b/docs/kafka-connect/administration.md new file mode 100644 index 0000000000000..8aceb854fe26f --- /dev/null +++ b/docs/kafka-connect/administration.md @@ -0,0 +1,58 @@ +--- +title: Administration +description: Administration +weight: 4 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +Kafka Connect's REST layer provides a set of APIs to enable administration of the cluster. This includes APIs to view the configuration of connectors and the status of their tasks, as well as to alter their current behavior (e.g. changing configuration and restarting tasks). + +When a connector is first submitted to the cluster, a rebalance is triggered between the Connect workers in order to distribute the load that consists of the tasks of the new connector. This same rebalancing procedure is also used when connectors increase or decrease the number of tasks they require, when a connector's configuration is changed, or when a worker is added or removed from the group as part of an intentional upgrade of the Connect cluster or due to a failure. + +In versions prior to 2.3.0, the Connect workers would rebalance the full set of connectors and their tasks in the cluster as a simple way to make sure that each worker has approximately the same amount of work. This behavior can be still enabled by setting `connect.protocol=eager`. + +Starting with 2.3.0, Kafka Connect is using by default a protocol that performs [incremental cooperative rebalancing](https://cwiki.apache.org/confluence/display/KAFKA/KIP-415%3A+Incremental+Cooperative+Rebalancing+in+Kafka+Connect) that incrementally balances the connectors and tasks across the Connect workers, affecting only tasks that are new, to be removed, or need to move from one worker to another. Other tasks are not stopped and restarted during the rebalance, as they would have been with the old protocol. + +If a Connect worker leaves the group, intentionally or due to a failure, Connect waits for `scheduled.rebalance.max.delay.ms` before triggering a rebalance. This delay defaults to five minutes (`300000ms`) to tolerate failures or upgrades of workers without immediately redistributing the load of a departing worker. If this worker returns within the configured delay, it gets its previously assigned tasks in full. However, this means that the tasks will remain unassigned until the time specified by `scheduled.rebalance.max.delay.ms` elapses. If a worker does not return within that time limit, Connect will reassign those tasks among the remaining workers in the Connect cluster. + +The new Connect protocol is enabled when all the workers that form the Connect cluster are configured with `connect.protocol=compatible`, which is also the default value when this property is missing. Therefore, upgrading to the new Connect protocol happens automatically when all the workers upgrade to 2.3.0. A rolling upgrade of the Connect cluster will activate incremental cooperative rebalancing when the last worker joins on version 2.3.0. + +You can use the REST API to view the current status of a connector and its tasks, including the ID of the worker to which each was assigned. For example, the `GET /connectors/file-source/status` request shows the status of a connector named `file-source`: + + + { + "name": "file-source", + "connector": { + "state": "RUNNING", + "worker_id": "192.168.1.208:8083" + }, + "tasks": [ + { + "id": 0, + "state": "RUNNING", + "worker_id": "192.168.1.209:8083" + } + ] + } + +Connectors and their tasks publish status updates to a shared topic (configured with `status.storage.topic`) which all workers in the cluster monitor. Because the workers consume this topic asynchronously, there is typically a (short) delay before a state change is visible through the status API. The following states are possible for a connector or one of its tasks: + + * **UNASSIGNED:** The connector/task has not yet been assigned to a worker. + * **RUNNING:** The connector/task is running. + * **PAUSED:** The connector/task has been administratively paused. + * **STOPPED:** The connector has been stopped. Note that this state is not applicable to tasks because the tasks for a stopped connector are shut down and won't be visible in the status API. + * **FAILED:** The connector/task has failed (usually by raising an exception, which is reported in the status output). + * **RESTARTING:** The connector/task is either actively restarting or is expected to restart soon + + + +In most cases, connector and task states will match, though they may be different for short periods of time when changes are occurring or if tasks have failed. For example, when a connector is first started, there may be a noticeable delay before the connector and its tasks have all transitioned to the RUNNING state. States will also diverge when tasks fail since Connect does not automatically restart failed tasks. To restart a connector/task manually, you can use the restart APIs listed above. Note that if you try to restart a task while a rebalance is taking place, Connect will return a 409 (Conflict) status code. You can retry after the rebalance completes, but it might not be necessary since rebalances effectively restart all the connectors and tasks in the cluster. + +Starting with 2.5.0, Kafka Connect uses the `status.storage.topic` to also store information related to the topics that each connector is using. Connect Workers use these per-connector topic status updates to respond to requests to the REST endpoint `GET /connectors/{name}/topics` by returning the set of topic names that a connector is using. A request to the REST endpoint `PUT /connectors/{name}/topics/reset` resets the set of active topics for a connector and allows a new set to be populated, based on the connector's latest pattern of topic usage. Upon connector deletion, the set of the connector's active topics is also deleted. Topic tracking is enabled by default but can be disabled by setting `topic.tracking.enable=false`. If you want to disallow requests to reset the active topics of connectors during runtime, set the Worker property `topic.tracking.allow.reset=false`. + +It's sometimes useful to temporarily stop the message processing of a connector. For example, if the remote system is undergoing maintenance, it would be preferable for source connectors to stop polling it for new data instead of filling logs with exception spam. For this use case, Connect offers a pause/resume API. While a source connector is paused, Connect will stop polling it for additional records. While a sink connector is paused, Connect will stop pushing new messages to it. The pause state is persistent, so even if you restart the cluster, the connector will not begin message processing again until the task has been resumed. Note that there may be a delay before all of a connector's tasks have transitioned to the PAUSED state since it may take time for them to finish whatever processing they were in the middle of when being paused. Additionally, failed tasks will not transition to the PAUSED state until they have been restarted. + +In 3.5.0, Connect introduced a stop API that completely shuts down the tasks for a connector and deallocates any resources claimed by them. This is different from pausing a connector where tasks are left idling and any resources claimed by them are left allocated (which allows the connector to begin processing data quickly once it is resumed). Stopping a connector is more efficient from a resource usage standpoint than pausing it, but can cause it to take longer to begin processing data once resumed. Note that the offsets for a connector can be only modified via the offsets management endpoints if it is in the stopped state. diff --git a/docs/kafka-connect/connector-development-guide.md b/docs/kafka-connect/connector-development-guide.md new file mode 100644 index 0000000000000..cb0f875437803 --- /dev/null +++ b/docs/kafka-connect/connector-development-guide.md @@ -0,0 +1,396 @@ +--- +title: Connector Development Guide +description: Connector Development Guide +weight: 3 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +This guide describes how developers can write new connectors for Kafka Connect to move data between Kafka and other systems. It briefly reviews a few key concepts and then describes how to create a simple connector. + +## Core Concepts and APIs + +### Connectors and Tasks + +To copy data between Kafka and another system, users create a `Connector` for the system they want to pull data from or push data to. Connectors come in two flavors: `SourceConnectors` import data from another system (e.g. `JDBCSourceConnector` would import a relational database into Kafka) and `SinkConnectors` export data (e.g. `HDFSSinkConnector` would export the contents of a Kafka topic to an HDFS file). + +`Connectors` do not perform any data copying themselves: their configuration describes the data to be copied, and the `Connector` is responsible for breaking that job into a set of `Tasks` that can be distributed to workers. These `Tasks` also come in two corresponding flavors: `SourceTask` and `SinkTask`. + +With an assignment in hand, each `Task` must copy its subset of the data to or from Kafka. In Kafka Connect, it should always be possible to frame these assignments as a set of input and output streams consisting of records with consistent schemas. Sometimes this mapping is obvious: each file in a set of log files can be considered a stream with each parsed line forming a record using the same schema and offsets stored as byte offsets in the file. In other cases it may require more effort to map to this model: a JDBC connector can map each table to a stream, but the offset is less clear. One possible mapping uses a timestamp column to generate queries incrementally returning new data, and the last queried timestamp can be used as the offset. + +### Streams and Records + +Each stream should be a sequence of key-value records. Both the keys and values can have complex structure -- many primitive types are provided, but arrays, objects, and nested data structures can be represented as well. The runtime data format does not assume any particular serialization format; this conversion is handled internally by the framework. + +In addition to the key and value, records (both those generated by sources and those delivered to sinks) have associated stream IDs and offsets. These are used by the framework to periodically commit the offsets of data that have been processed so that in the event of failures, processing can resume from the last committed offsets, avoiding unnecessary reprocessing and duplication of events. + +### Dynamic Connectors + +Not all jobs are static, so `Connector` implementations are also responsible for monitoring the external system for any changes that might require reconfiguration. For example, in the `JDBCSourceConnector` example, the `Connector` might assign a set of tables to each `Task`. When a new table is created, it must discover this so it can assign the new table to one of the `Tasks` by updating its configuration. When it notices a change that requires reconfiguration (or a change in the number of `Tasks`), it notifies the framework and the framework updates any corresponding `Tasks`. + +## Developing a Simple Connector + +Developing a connector only requires implementing two interfaces, the `Connector` and `Task`. A simple example is included with the source code for Kafka in the `file` package. This connector is meant for use in standalone mode and has implementations of a `SourceConnector`/`SourceTask` to read each line of a file and emit it as a record and a `SinkConnector`/`SinkTask` that writes each record to a file. + +The rest of this section will walk through some code to demonstrate the key steps in creating a connector, but developers should also refer to the full example source code as many details are omitted for brevity. + +### Connector Example + +We'll cover the `SourceConnector` as a simple example. `SinkConnector` implementations are very similar. Pick a package and class name, these examples will use the `FileStreamSourceConnector` but substitute your own class name where appropriate. In order to make the plugin discoverable at runtime, add a ServiceLoader manifest to your resources in `META-INF/services/org.apache.kafka.connect.source.SourceConnector` with your fully-qualified class name on a single line: + + + com.example.FileStreamSourceConnector + +Create a class that inherits from `SourceConnector` and add a field that will store the configuration information to be propagated to the task(s) (the topic to send data to, and optionally - the filename to read from and the maximum batch size): + + + package com.example; + + public class FileStreamSourceConnector extends SourceConnector { + private Map props; + +The easiest method to fill in is `taskClass()`, which defines the class that should be instantiated in worker processes to actually read the data: + + + @Override + public Class taskClass() { + return FileStreamSourceTask.class; + } + +We will define the `FileStreamSourceTask` class below. Next, we add some standard lifecycle methods, `start()` and `stop()`: + + + @Override + public void start(Map props) { + // Initialization logic and setting up of resources can take place in this method. + // This connector doesn't need to do any of that, but we do log a helpful message to the user. + + this.props = props; + AbstractConfig config = new AbstractConfig(CONFIG_DEF, props); + String filename = config.getString(FILE_CONFIG); + filename = (filename == null || filename.isEmpty()) ? "standard input" : config.getString(FILE_CONFIG); + log.info("Starting file source connector reading from {}", filename); + } + + @Override + public void stop() { + // Nothing to do since no background monitoring is required. + } + +Finally, the real core of the implementation is in `taskConfigs()`. In this case we are only handling a single file, so even though we may be permitted to generate more tasks as per the `maxTasks` argument, we return a list with only one entry: + + + @Override + public List> taskConfigs(int maxTasks) { + // Note that the task configs could contain configs additional to or different from the connector configs if needed. For instance, + // if different tasks have different responsibilities, or if different tasks are meant to process different subsets of the source data stream). + ArrayList> configs = new ArrayList<>(); + // Only one input stream makes sense. + configs.add(props); + return configs; + } + +Even with multiple tasks, this method implementation is usually pretty simple. It just has to determine the number of input tasks, which may require contacting the remote service it is pulling data from, and then divvy them up. Because some patterns for splitting work among tasks are so common, some utilities are provided in `ConnectorUtils` to simplify these cases. + +Note that this simple example does not include dynamic input. See the discussion in the next section for how to trigger updates to task configs. + +### Task Example - Source Task + +Next we'll describe the implementation of the corresponding `SourceTask`. The implementation is short, but too long to cover completely in this guide. We'll use pseudo-code to describe most of the implementation, but you can refer to the source code for the full example. + +Just as with the connector, we need to create a class inheriting from the appropriate base `Task` class. It also has some standard lifecycle methods: + + + public class FileStreamSourceTask extends SourceTask { + private String filename; + private InputStream stream; + private String topic; + private int batchSize; + + @Override + public void start(Map props) { + filename = props.get(FileStreamSourceConnector.FILE_CONFIG); + stream = openOrThrowError(filename); + topic = props.get(FileStreamSourceConnector.TOPIC_CONFIG); + batchSize = props.get(FileStreamSourceConnector.TASK_BATCH_SIZE_CONFIG); + } + + @Override + public synchronized void stop() { + stream.close(); + } + +These are slightly simplified versions, but show that these methods should be relatively simple and the only work they should perform is allocating or freeing resources. There are two points to note about this implementation. First, the `start()` method does not yet handle resuming from a previous offset, which will be addressed in a later section. Second, the `stop()` method is synchronized. This will be necessary because `SourceTasks` are given a dedicated thread which they can block indefinitely, so they need to be stopped with a call from a different thread in the Worker. + +Next, we implement the main functionality of the task, the `poll()` method which gets events from the input system and returns a `List`: + + + @Override + public List poll() throws InterruptedException { + try { + ArrayList records = new ArrayList<>(); + while (streamValid(stream) && records.isEmpty()) { + LineAndOffset line = readToNextLine(stream); + if (line != null) { + Map sourcePartition = Collections.singletonMap("filename", filename); + Map sourceOffset = Collections.singletonMap("position", streamOffset); + records.add(new SourceRecord(sourcePartition, sourceOffset, topic, Schema.STRING_SCHEMA, line)); + if (records.size() >= batchSize) { + return records; + } + } else { + Thread.sleep(1); + } + } + return records; + } catch (IOException e) { + // Underlying stream was killed, probably as a result of calling stop. Allow to return + // null, and driving thread will handle any shutdown if necessary. + } + return null; + } + +Again, we've omitted some details, but we can see the important steps: the `poll()` method is going to be called repeatedly, and for each call it will loop trying to read records from the file. For each line it reads, it also tracks the file offset. It uses this information to create an output `SourceRecord` with four pieces of information: the source partition (there is only one, the single file being read), source offset (byte offset in the file), output topic name, and output value (the line, and we include a schema indicating this value will always be a string). Other variants of the `SourceRecord` constructor can also include a specific output partition, a key, and headers. + +Note that this implementation uses the normal Java `InputStream` interface and may sleep if data is not available. This is acceptable because Kafka Connect provides each task with a dedicated thread. While task implementations have to conform to the basic `poll()` interface, they have a lot of flexibility in how they are implemented. In this case, an NIO-based implementation would be more efficient, but this simple approach works, is quick to implement, and is compatible with older versions of Java. + +Although not used in the example, `SourceTask` also provides two APIs to commit offsets in the source system: `commit` and `commitRecord`. The APIs are provided for source systems which have an acknowledgement mechanism for messages. Overriding these methods allows the source connector to acknowledge messages in the source system, either in bulk or individually, once they have been written to Kafka. The `commit` API stores the offsets in the source system, up to the offsets that have been returned by `poll`. The implementation of this API should block until the commit is complete. The `commitRecord` API saves the offset in the source system for each `SourceRecord` after it is written to Kafka. As Kafka Connect will record offsets automatically, `SourceTask`s are not required to implement them. In cases where a connector does need to acknowledge messages in the source system, only one of the APIs is typically required. + +### Sink Tasks + +The previous section described how to implement a simple `SourceTask`. Unlike `SourceConnector` and `SinkConnector`, `SourceTask` and `SinkTask` have very different interfaces because `SourceTask` uses a pull interface and `SinkTask` uses a push interface. Both share the common lifecycle methods, but the `SinkTask` interface is quite different: + + + public abstract class SinkTask implements Task { + public void initialize(SinkTaskContext context) { + this.context = context; + } + + public abstract void put(Collection records); + + public void flush(Map currentOffsets) { + } + +The `SinkTask` documentation contains full details, but this interface is nearly as simple as the `SourceTask`. The `put()` method should contain most of the implementation, accepting sets of `SinkRecords`, performing any required translation, and storing them in the destination system. This method does not need to ensure the data has been fully written to the destination system before returning. In fact, in many cases internal buffering will be useful so an entire batch of records can be sent at once, reducing the overhead of inserting events into the downstream data store. The `SinkRecords` contain essentially the same information as `SourceRecords`: Kafka topic, partition, offset, the event key and value, and optional headers. + +The `flush()` method is used during the offset commit process, which allows tasks to recover from failures and resume from a safe point such that no events will be missed. The method should push any outstanding data to the destination system and then block until the write has been acknowledged. The `offsets` parameter can often be ignored, but is useful in some cases where implementations want to store offset information in the destination store to provide exactly-once delivery. For example, an HDFS connector could do this and use atomic move operations to make sure the `flush()` operation atomically commits the data and offsets to a final location in HDFS. + +### [Errant Record Reporter](connect_errantrecordreporter) + +When error reporting is enabled for a connector, the connector can use an `ErrantRecordReporter` to report problems with individual records sent to a sink connector. The following example shows how a connector's `SinkTask` subclass might obtain and use the `ErrantRecordReporter`, safely handling a null reporter when the DLQ is not enabled or when the connector is installed in an older Connect runtime that doesn't have this reporter feature: + + + private ErrantRecordReporter reporter; + + @Override + public void start(Map props) { + ... + try { + reporter = context.errantRecordReporter(); // may be null if DLQ not enabled + } catch (NoSuchMethodException | NoClassDefFoundError e) { + // Will occur in Connect runtimes earlier than 2.6 + reporter = null; + } + } + + @Override + public void put(Collection records) { + for (SinkRecord record: records) { + try { + // attempt to process and send record to data sink + process(record); + } catch(Exception e) { + if (reporter != null) { + // Send errant record to error reporter + reporter.report(record, e); + } else { + // There's no error reporter, so fail + throw new ConnectException("Failed on record", e); + } + } + } + } + +### Resuming from Previous Offsets + +The `SourceTask` implementation included a stream ID (the input filename) and offset (position in the file) with each record. The framework uses this to commit offsets periodically so that in the case of a failure, the task can recover and minimize the number of events that are reprocessed and possibly duplicated (or to resume from the most recent offset if Kafka Connect was stopped gracefully, e.g. in standalone mode or due to a job reconfiguration). This commit process is completely automated by the framework, but only the connector knows how to seek back to the right position in the input stream to resume from that location. + +To correctly resume upon startup, the task can use the `SourceContext` passed into its `initialize()` method to access the offset data. In `initialize()`, we would add a bit more code to read the offset (if it exists) and seek to that position: + + + stream = new FileInputStream(filename); + Map offset = context.offsetStorageReader().offset(Collections.singletonMap(FILENAME_FIELD, filename)); + if (offset != null) { + Long lastRecordedOffset = (Long) offset.get("position"); + if (lastRecordedOffset != null) + seekToOffset(stream, lastRecordedOffset); + } + +Of course, you might need to read many keys for each of the input streams. The `OffsetStorageReader` interface also allows you to issue bulk reads to efficiently load all offsets, then apply them by seeking each input stream to the appropriate position. + +### Exactly-once source connectors + +#### Supporting exactly-once + +With the passing of [KIP-618](https://cwiki.apache.org/confluence/display/KAFKA/KIP-618%3A+Exactly-Once+Support+for+Source+Connectors), Kafka Connect supports exactly-once source connectors as of version 3.3.0. In order for a source connector to take advantage of this support, it must be able to provide meaningful source offsets for each record that it emits, and resume consumption from the external system at the exact position corresponding to any of those offsets without dropping or duplicating messages. + +#### Defining transaction boundaries + +By default, the Kafka Connect framework will create and commit a new Kafka transaction for each batch of records that a source task returns from its `poll` method. However, connectors can also define their own transaction boundaries, which can be enabled by users by setting the `transaction.boundary` property to `connector` in the config for the connector. + +If enabled, the connector's tasks will have access to a `TransactionContext` from their `SourceTaskContext`, which they can use to control when transactions are aborted and committed. + +For example, to commit a transaction at least every ten records: + + + private int recordsSent; + + @Override + public void start(Map props) { + this.recordsSent = 0; + } + + @Override + public List poll() { + List records = fetchRecords(); + boolean shouldCommit = false; + for (SourceRecord record : records) { + if (++this.recordsSent >= 10) { + shouldCommit = true; + } + } + if (shouldCommit) { + this.recordsSent = 0; + this.context.transactionContext().commitTransaction(); + } + return records; + } + + +Or to commit a transaction for exactly every tenth record: + + + private int recordsSent; + + @Override + public void start(Map props) { + this.recordsSent = 0; + } + + @Override + public List poll() { + List records = fetchRecords(); + for (SourceRecord record : records) { + if (++this.recordsSent % 10 == 0) { + this.context.transactionContext().commitTransaction(record); + } + } + return records; + } + + +Most connectors do not need to define their own transaction boundaries. However, it may be useful if files or objects in the source system are broken up into multiple source records, but should be delivered atomically. Additionally, it may be useful if it is impossible to give each source record a unique source offset, if every record with a given offset is delivered within a single transaction. + +Note that if the user has not enabled connector-defined transaction boundaries in the connector configuration, the `TransactionContext` returned by `context.transactionContext()` will be `null`. + +#### Validation APIs + +A few additional preflight validation APIs can be implemented by source connector developers. + +Some users may require exactly-once semantics from a connector. In this case, they may set the `exactly.once.support` property to `required` in the configuration for the connector. When this happens, the Kafka Connect framework will ask the connector whether it can provide exactly-once semantics with the specified configuration. This is done by invoking the `exactlyOnceSupport` method on the connector. + +If a connector doesn't support exactly-once semantics, it should still implement this method to let users know for certain that it cannot provide exactly-once semantics: + + + @Override + public ExactlyOnceSupport exactlyOnceSupport(Map props) { + // This connector cannot provide exactly-once semantics under any conditions + return ExactlyOnceSupport.UNSUPPORTED; + } + + +Otherwise, a connector should examine the configuration, and return `ExactlyOnceSupport.SUPPORTED` if it can provide exactly-once semantics: + + + @Override + public ExactlyOnceSupport exactlyOnceSupport(Map props) { + // This connector can always provide exactly-once semantics + return ExactlyOnceSupport.SUPPORTED; + } + + +Additionally, if the user has configured the connector to define its own transaction boundaries, the Kafka Connect framework will ask the connector whether it can define its own transaction boundaries with the specified configuration, using the `canDefineTransactionBoundaries` method: + + + @Override + public ConnectorTransactionBoundaries canDefineTransactionBoundaries(Map props) { + // This connector can always define its own transaction boundaries + return ConnectorTransactionBoundaries.SUPPORTED; + } + + +This method should only be implemented for connectors that can define their own transaction boundaries in some cases. If a connector is never able to define its own transaction boundaries, it does not need to implement this method. + +## Dynamic Input/Output Streams + +Kafka Connect is intended to define bulk data copying jobs, such as copying an entire database rather than creating many jobs to copy each table individually. One consequence of this design is that the set of input or output streams for a connector can vary over time. + +Source connectors need to monitor the source system for changes, e.g. table additions/deletions in a database. When they pick up changes, they should notify the framework via the `ConnectorContext` object that reconfiguration is necessary. For example, in a `SourceConnector`: + + + if (inputsChanged()) + this.context.requestTaskReconfiguration(); + +The framework will promptly request new configuration information and update the tasks, allowing them to gracefully commit their progress before reconfiguring them. Note that in the `SourceConnector` this monitoring is currently left up to the connector implementation. If an extra thread is required to perform this monitoring, the connector must allocate it itself. + +Ideally this code for monitoring changes would be isolated to the `Connector` and tasks would not need to worry about them. However, changes can also affect tasks, most commonly when one of their input streams is destroyed in the input system, e.g. if a table is dropped from a database. If the `Task` encounters the issue before the `Connector`, which will be common if the `Connector` needs to poll for changes, the `Task` will need to handle the subsequent error. Thankfully, this can usually be handled simply by catching and handling the appropriate exception. + +`SinkConnectors` usually only have to handle the addition of streams, which may translate to new entries in their outputs (e.g., a new database table). The framework manages any changes to the Kafka input, such as when the set of input topics changes because of a regex subscription. `SinkTasks` should expect new input streams, which may require creating new resources in the downstream system, such as a new table in a database. The trickiest situation to handle in these cases may be conflicts between multiple `SinkTasks` seeing a new input stream for the first time and simultaneously trying to create the new resource. `SinkConnectors`, on the other hand, will generally require no special code for handling a dynamic set of streams. + +## Configuration Validation + +Kafka Connect allows you to validate connector configurations before submitting a connector to be executed and can provide feedback about errors and recommended values. To take advantage of this, connector developers need to provide an implementation of `config()` to expose the configuration definition to the framework. + +The following code in `FileStreamSourceConnector` defines the configuration and exposes it to the framework. + + + static final ConfigDef CONFIG_DEF = new ConfigDef() + .define(FILE_CONFIG, Type.STRING, null, Importance.HIGH, "Source filename. If not specified, the standard input will be used") + .define(TOPIC_CONFIG, Type.STRING, ConfigDef.NO_DEFAULT_VALUE, new ConfigDef.NonEmptyString(), Importance.HIGH, "The topic to publish data to") + .define(TASK_BATCH_SIZE_CONFIG, Type.INT, DEFAULT_TASK_BATCH_SIZE, Importance.LOW, + "The maximum number of records the source task can read from the file each time it is polled"); + + public ConfigDef config() { + return CONFIG_DEF; + } + +`ConfigDef` class is used for specifying the set of expected configurations. For each configuration, you can specify the name, the type, the default value, the documentation, the group information, the order in the group, the width of the configuration value and the name suitable for display in the UI. Plus, you can provide special validation logic used for single configuration validation by overriding the `Validator` class. Moreover, as there may be dependencies between configurations, for example, the valid values and visibility of a configuration may change according to the values of other configurations. To handle this, `ConfigDef` allows you to specify the dependents of a configuration and to provide an implementation of `Recommender` to get valid values and set visibility of a configuration given the current configuration values. + +Also, the `validate()` method in `Connector` provides a default validation implementation which returns a list of allowed configurations together with configuration errors and recommended values for each configuration. However, it does not use the recommended values for configuration validation. You may provide an override of the default implementation for customized configuration validation, which may use the recommended values. + +## Working with Schemas + +The FileStream connectors are good examples because they are simple, but they also have trivially structured data -- each line is just a string. Almost all practical connectors will need schemas with more complex data formats. + +To create more complex data, you'll need to work with the Kafka Connect `data` API. Most structured records will need to interact with two classes in addition to primitive types: `Schema` and `Struct`. + +The API documentation provides a complete reference, but here is a simple example creating a `Schema` and `Struct`: + + + Schema schema = SchemaBuilder.struct().name(NAME) + .field("name", Schema.STRING_SCHEMA) + .field("age", Schema.INT_SCHEMA) + .field("admin", SchemaBuilder.bool().defaultValue(false).build()) + .build(); + + Struct struct = new Struct(schema) + .put("name", "Barbara Liskov") + .put("age", 75); + +If you are implementing a source connector, you'll need to decide when and how to create schemas. Where possible, you should avoid recomputing them as much as possible. For example, if your connector is guaranteed to have a fixed schema, create it statically and reuse a single instance. + +However, many connectors will have dynamic schemas. One simple example of this is a database connector. Considering even just a single table, the schema will not be predefined for the entire connector (as it varies from table to table). But it also may not be fixed for a single table over the lifetime of the connector since the user may execute an `ALTER TABLE` command. The connector must be able to detect these changes and react appropriately. + +Sink connectors are usually simpler because they are consuming data and therefore do not need to create schemas. However, they should take just as much care to validate that the schemas they receive have the expected format. When the schema does not match -- usually indicating the upstream producer is generating invalid data that cannot be correctly translated to the destination system -- sink connectors should throw an exception to indicate this error to the system. diff --git a/docs/kafka-connect/overview.md b/docs/kafka-connect/overview.md new file mode 100644 index 0000000000000..4541b21e1401b --- /dev/null +++ b/docs/kafka-connect/overview.md @@ -0,0 +1,22 @@ +--- +title: Overview +description: Overview +weight: 1 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +Kafka Connect is a tool for scalably and reliably streaming data between Apache Kafka and other systems. It makes it simple to quickly define _connectors_ that move large collections of data into and out of Kafka. Kafka Connect can ingest entire databases or collect metrics from all your application servers into Kafka topics, making the data available for stream processing with low latency. An export job can deliver data from Kafka topics into secondary storage and query systems or into batch systems for offline analysis. + +Kafka Connect features include: + + * **A common framework for Kafka connectors** \- Kafka Connect standardizes integration of other data systems with Kafka, simplifying connector development, deployment, and management + * **Distributed and standalone modes** \- scale up to a large, centrally managed service supporting an entire organization or scale down to development, testing, and small production deployments + * **REST interface** \- submit and manage connectors to your Kafka Connect cluster via an easy to use REST API + * **Automatic offset management** \- with just a little information from connectors, Kafka Connect can manage the offset commit process automatically so connector developers do not need to worry about this error prone part of connector development + * **Distributed and scalable by default** \- Kafka Connect builds on the existing group management protocol. More workers can be added to scale up a Kafka Connect cluster. + * **Streaming/batch integration** \- leveraging Kafka's existing capabilities, Kafka Connect is an ideal solution for bridging streaming and batch data systems + + diff --git a/docs/kafka-connect/user-guide.md b/docs/kafka-connect/user-guide.md new file mode 100644 index 0000000000000..72d974930295d --- /dev/null +++ b/docs/kafka-connect/user-guide.md @@ -0,0 +1,500 @@ +--- +title: User Guide +description: User Guide +weight: 2 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +The [quickstart](../quickstart) provides a brief example of how to run a standalone version of Kafka Connect. This section describes how to configure, run, and manage Kafka Connect in more detail. + +## Running Kafka Connect + +Kafka Connect currently supports two modes of execution: standalone (single process) and distributed. + +In standalone mode all work is performed in a single process. This configuration is simpler to setup and get started with and may be useful in situations where only one worker makes sense (e.g. collecting log files), but it does not benefit from some of the features of Kafka Connect such as fault tolerance. You can start a standalone process with the following command: + + + > bin/connect-standalone.sh config/connect-standalone.properties [connector1.properties connector2.json ...] + +The first parameter is the configuration for the worker. This includes settings such as the Kafka connection parameters, serialization format, and how frequently to commit offsets. The provided example should work well with a local cluster running with the default configuration provided by `config/server.properties`. It will require tweaking to use with a different configuration or production deployment. All workers (both standalone and distributed) require a few configs: + + * `bootstrap.servers` \- List of Kafka servers used to bootstrap connections to Kafka + * `key.converter` \- Converter class used to convert between Kafka Connect format and the serialized form that is written to Kafka. This controls the format of the keys in messages written to or read from Kafka, and since this is independent of connectors it allows any connector to work with any serialization format. Examples of common formats include JSON and Avro. + * `value.converter` \- Converter class used to convert between Kafka Connect format and the serialized form that is written to Kafka. This controls the format of the values in messages written to or read from Kafka, and since this is independent of connectors it allows any connector to work with any serialization format. Examples of common formats include JSON and Avro. + * `plugin.path` (default `empty`) - a list of paths that contain Connect plugins (connectors, converters, transformations). Before running quick starts, users must add the absolute path that contains the example FileStreamSourceConnector and FileStreamSinkConnector packaged in `connect-file-"version".jar`, because these connectors are not included by default to the `CLASSPATH` or the `plugin.path` of the Connect worker (see plugin.path property for examples). + + + +The important configuration options specific to standalone mode are: + + * `offset.storage.file.filename` \- File to store source connector offsets + + + +The parameters that are configured here are intended for producers and consumers used by Kafka Connect to access the configuration, offset and status topics. For configuration of the producers used by Kafka source tasks and the consumers used by Kafka sink tasks, the same parameters can be used but need to be prefixed with `producer.` and `consumer.` respectively. The only Kafka client parameter that is inherited without a prefix from the worker configuration is `bootstrap.servers`, which in most cases will be sufficient, since the same cluster is often used for all purposes. A notable exception is a secured cluster, which requires extra parameters to allow connections. These parameters will need to be set up to three times in the worker configuration, once for management access, once for Kafka sources and once for Kafka sinks. + +Starting with 2.3.0, client configuration overrides can be configured individually per connector by using the prefixes `producer.override.` and `consumer.override.` for Kafka sources or Kafka sinks respectively. These overrides are included with the rest of the connector's configuration properties. + +The remaining parameters are connector configuration files. Each file may either be a Java Properties file or a JSON file containing an object with the same structure as the request body of either the `POST /connectors` endpoint or the `PUT /connectors/{name}/config` endpoint (see the [OpenAPI documentation](/37/generated/connect_rest.yaml)). You may include as many as you want, but all will execute within the same process (on different threads). You can also choose not to specify any connector configuration files on the command line, and instead use the REST API to create connectors at runtime after your standalone worker starts. + +Distributed mode handles automatic balancing of work, allows you to scale up (or down) dynamically, and offers fault tolerance both in the active tasks and for configuration and offset commit data. Execution is very similar to standalone mode: + + + > bin/connect-distributed.sh config/connect-distributed.properties + +The difference is in the class which is started and the configuration parameters which change how the Kafka Connect process decides where to store configurations, how to assign work, and where to store offsets and task statues. In the distributed mode, Kafka Connect stores the offsets, configs and task statuses in Kafka topics. It is recommended to manually create the topics for offset, configs and statuses in order to achieve the desired the number of partitions and replication factors. If the topics are not yet created when starting Kafka Connect, the topics will be auto created with default number of partitions and replication factor, which may not be best suited for its usage. + +In particular, the following configuration parameters, in addition to the common settings mentioned above, are critical to set before starting your cluster: + + * `group.id` (default `connect-cluster`) - unique name for the cluster, used in forming the Connect cluster group; note that this **must not conflict** with consumer group IDs + * `config.storage.topic` (default `connect-configs`) - topic to use for storing connector and task configurations; note that this should be a single partition, highly replicated, compacted topic. You may need to manually create the topic to ensure the correct configuration as auto created topics may have multiple partitions or be automatically configured for deletion rather than compaction + * `offset.storage.topic` (default `connect-offsets`) - topic to use for storing offsets; this topic should have many partitions, be replicated, and be configured for compaction + * `status.storage.topic` (default `connect-status`) - topic to use for storing statuses; this topic can have multiple partitions, and should be replicated and configured for compaction + + + +Note that in distributed mode the connector configurations are not passed on the command line. Instead, use the REST API described below to create, modify, and destroy connectors. + +## Configuring Connectors + +Connector configurations are simple key-value mappings. In both standalone and distributed mode, they are included in the JSON payload for the REST request that creates (or modifies) the connector. In standalone mode these can also be defined in a properties file and passed to the Connect process on the command line. + +Most configurations are connector dependent, so they can't be outlined here. However, there are a few common options: + + * `name` \- Unique name for the connector. Attempting to register again with the same name will fail. + * `connector.class` \- The Java class for the connector + * `tasks.max` \- The maximum number of tasks that should be created for this connector. The connector may create fewer tasks if it cannot achieve this level of parallelism. + * `key.converter` \- (optional) Override the default key converter set by the worker. + * `value.converter` \- (optional) Override the default value converter set by the worker. + + + +The `connector.class` config supports several formats: the full name or alias of the class for this connector. If the connector is org.apache.kafka.connect.file.FileStreamSinkConnector, you can either specify this full name or use FileStreamSink or FileStreamSinkConnector to make the configuration a bit shorter. + +Sink connectors also have a few additional options to control their input. Each sink connector must set one of the following: + + * `topics` \- A comma-separated list of topics to use as input for this connector + * `topics.regex` \- A Java regular expression of topics to use as input for this connector + + + +For any other options, you should consult the documentation for the connector. + +## Transformations + +Connectors can be configured with transformations to make lightweight message-at-a-time modifications. They can be convenient for data massaging and event routing. + +A transformation chain can be specified in the connector configuration. + + * `transforms` \- List of aliases for the transformation, specifying the order in which the transformations will be applied. + * `transforms.$alias.type` \- Fully qualified class name for the transformation. + * `transforms.$alias.$transformationSpecificConfig` Configuration properties for the transformation + + + +For example, lets take the built-in file source connector and use a transformation to add a static field. + +Throughout the example we'll use schemaless JSON data format. To use schemaless format, we changed the following two lines in `connect-standalone.properties` from true to false: + + + key.converter.schemas.enable + value.converter.schemas.enable + +The file source connector reads each line as a String. We will wrap each line in a Map and then add a second field to identify the origin of the event. To do this, we use two transformations: + + * **HoistField** to place the input line inside a Map + * **InsertField** to add the static field. In this example we'll indicate that the record came from a file connector + + + +After adding the transformations, `connect-file-source.properties` file looks as following: + + + name=local-file-source + connector.class=FileStreamSource + tasks.max=1 + file=test.txt + topic=connect-test + transforms=MakeMap, InsertSource + transforms.MakeMap.type=org.apache.kafka.connect.transforms.HoistField$Value + transforms.MakeMap.field=line + transforms.InsertSource.type=org.apache.kafka.connect.transforms.InsertField$Value + transforms.InsertSource.static.field=data_source + transforms.InsertSource.static.value=test-file-source + +All the lines starting with `transforms` were added for the transformations. You can see the two transformations we created: "InsertSource" and "MakeMap" are aliases that we chose to give the transformations. The transformation types are based on the list of built-in transformations you can see below. Each transformation type has additional configuration: HoistField requires a configuration called "field", which is the name of the field in the map that will include the original String from the file. InsertField transformation lets us specify the field name and the value that we are adding. + +When we ran the file source connector on my sample file without the transformations, and then read them using `kafka-console-consumer.sh`, the results were: + + + "foo" + "bar" + "hello world" + +We then create a new file connector, this time after adding the transformations to the configuration file. This time, the results will be: + + + {"line":"foo","data_source":"test-file-source"} + {"line":"bar","data_source":"test-file-source"} + {"line":"hello world","data_source":"test-file-source"} + +You can see that the lines we've read are now part of a JSON map, and there is an extra field with the static value we specified. This is just one example of what you can do with transformations. + +### Included transformations + +Several widely-applicable data and routing transformations are included with Kafka Connect: + + * InsertField - Add a field using either static data or record metadata + * ReplaceField - Filter or rename fields + * MaskField - Replace field with valid null value for the type (0, empty string, etc) or custom replacement (non-empty string or numeric value only) + * ValueToKey - Replace the record key with a new key formed from a subset of fields in the record value + * HoistField - Wrap the entire event as a single field inside a Struct or a Map + * ExtractField - Extract a specific field from Struct and Map and include only this field in results + * SetSchemaMetadata - modify the schema name or version + * TimestampRouter - Modify the topic of a record based on original topic and timestamp. Useful when using a sink that needs to write to different tables or indexes based on timestamps + * RegexRouter - modify the topic of a record based on original topic, replacement string and a regular expression + * Filter - Removes messages from all further processing. This is used with a predicate to selectively filter certain messages. + * InsertHeader - Add a header using static data + * HeadersFrom - Copy or move fields in the key or value to the record headers + * DropHeaders - Remove headers by name + + + +Details on how to configure each transformation are listed below: + +{{< include-html file="/static/37/generated/connect_transforms.html" >}} + +### Predicates + +Transformations can be configured with predicates so that the transformation is applied only to messages which satisfy some condition. In particular, when combined with the **Filter** transformation predicates can be used to selectively filter out certain messages. + +Predicates are specified in the connector configuration. + + * `predicates` \- Set of aliases for the predicates to be applied to some of the transformations. + * `predicates.$alias.type` \- Fully qualified class name for the predicate. + * `predicates.$alias.$predicateSpecificConfig` \- Configuration properties for the predicate. + + + +All transformations have the implicit config properties `predicate` and `negate`. A predicular predicate is associated with a transformation by setting the transformation's `predicate` config to the predicate's alias. The predicate's value can be reversed using the `negate` configuration property. + +For example, suppose you have a source connector which produces messages to many different topics and you want to: + + * filter out the messages in the 'foo' topic entirely + * apply the ExtractField transformation with the field name 'other_field' to records in all topics _except_ the topic 'bar' + + + +To do this we need first to filter out the records destined for the topic 'foo'. The Filter transformation removes records from further processing, and can use the TopicNameMatches predicate to apply the transformation only to records in topics which match a certain regular expression. TopicNameMatches's only configuration property is `pattern` which is a Java regular expression for matching against the topic name. The configuration would look like this: + + + transforms=Filter + transforms.Filter.type=org.apache.kafka.connect.transforms.Filter + transforms.Filter.predicate=IsFoo + + predicates=IsFoo + predicates.IsFoo.type=org.apache.kafka.connect.transforms.predicates.TopicNameMatches + predicates.IsFoo.pattern=foo + +Next we need to apply ExtractField only when the topic name of the record is not 'bar'. We can't just use TopicNameMatches directly, because that would apply the transformation to matching topic names, not topic names which do _not_ match. The transformation's implicit `negate` config properties allows us to invert the set of records which a predicate matches. Adding the configuration for this to the previous example we arrive at: + + + transforms=Filter,Extract + transforms.Filter.type=org.apache.kafka.connect.transforms.Filter + transforms.Filter.predicate=IsFoo + + transforms.Extract.type=org.apache.kafka.connect.transforms.ExtractField$Key + transforms.Extract.field=other_field + transforms.Extract.predicate=IsBar + transforms.Extract.negate=true + + predicates=IsFoo,IsBar + predicates.IsFoo.type=org.apache.kafka.connect.transforms.predicates.TopicNameMatches + predicates.IsFoo.pattern=foo + + predicates.IsBar.type=org.apache.kafka.connect.transforms.predicates.TopicNameMatches + predicates.IsBar.pattern=bar + +Kafka Connect includes the following predicates: + + * `TopicNameMatches` \- matches records in a topic with a name matching a particular Java regular expression. + * `HasHeaderKey` \- matches records which have a header with the given key. + * `RecordIsTombstone` \- matches tombstone records, that is records with a null value. + + + +Details on how to configure each predicate are listed below: + +{{< include-html file="/static/37/generated/connect_predicates.html" >}} + +## REST API + +Since Kafka Connect is intended to be run as a service, it also provides a REST API for managing connectors. This REST API is available in both standalone and distributed mode. The REST API server can be configured using the `listeners` configuration option. This field should contain a list of listeners in the following format: `protocol://host:port,protocol2://host2:port2`. Currently supported protocols are `http` and `https`. For example: + + + listeners=http://localhost:8080,https://localhost:8443 + +By default, if no `listeners` are specified, the REST server runs on port 8083 using the HTTP protocol. When using HTTPS, the configuration has to include the SSL configuration. By default, it will use the `ssl.*` settings. In case it is needed to use different configuration for the REST API than for connecting to Kafka brokers, the fields can be prefixed with `listeners.https`. When using the prefix, only the prefixed options will be used and the `ssl.*` options without the prefix will be ignored. Following fields can be used to configure HTTPS for the REST API: + + * `ssl.keystore.location` + * `ssl.keystore.password` + * `ssl.keystore.type` + * `ssl.key.password` + * `ssl.truststore.location` + * `ssl.truststore.password` + * `ssl.truststore.type` + * `ssl.enabled.protocols` + * `ssl.provider` + * `ssl.protocol` + * `ssl.cipher.suites` + * `ssl.keymanager.algorithm` + * `ssl.secure.random.implementation` + * `ssl.trustmanager.algorithm` + * `ssl.endpoint.identification.algorithm` + * `ssl.client.auth` + + + +The REST API is used not only by users to monitor / manage Kafka Connect. In distributed mode, it is also used for the Kafka Connect cross-cluster communication. Some requests received on the follower nodes REST API will be forwarded to the leader node REST API. In case the URI under which is given host reachable is different from the URI which it listens on, the configuration options `rest.advertised.host.name`, `rest.advertised.port` and `rest.advertised.listener` can be used to change the URI which will be used by the follower nodes to connect with the leader. When using both HTTP and HTTPS listeners, the `rest.advertised.listener` option can be also used to define which listener will be used for the cross-cluster communication. When using HTTPS for communication between nodes, the same `ssl.*` or `listeners.https` options will be used to configure the HTTPS client. + +The following are the currently supported REST API endpoints: + + * `GET /connectors` \- return a list of active connectors + * `POST /connectors` \- create a new connector; the request body should be a JSON object containing a string `name` field and an object `config` field with the connector configuration parameters. The JSON object may also optionally contain a string `initial_state` field which can take the following values - `STOPPED`, `PAUSED` or `RUNNING` (the default value) + * `GET /connectors/{name}` \- get information about a specific connector + * `GET /connectors/{name}/config` \- get the configuration parameters for a specific connector + * `PUT /connectors/{name}/config` \- update the configuration parameters for a specific connector + * `GET /connectors/{name}/status` \- get current status of the connector, including if it is running, failed, paused, etc., which worker it is assigned to, error information if it has failed, and the state of all its tasks + * `GET /connectors/{name}/tasks` \- get a list of tasks currently running for a connector along with their configurations + * `GET /connectors/{name}/tasks-config` \- get the configuration of all tasks for a specific connector. This endpoint is deprecated and will be removed in the next major release. Please use the `GET /connectors/{name}/tasks` endpoint instead. Note that the response structures of the two endpoints differ slightly, please refer to the [OpenAPI documentation](/37/generated/connect_rest.yaml) for more details + * `GET /connectors/{name}/tasks/{taskid}/status` \- get current status of the task, including if it is running, failed, paused, etc., which worker it is assigned to, and error information if it has failed + * `PUT /connectors/{name}/pause` \- pause the connector and its tasks, which stops message processing until the connector is resumed. Any resources claimed by its tasks are left allocated, which allows the connector to begin processing data quickly once it is resumed. + * `PUT /connectors/{name}/stop` \- stop the connector and shut down its tasks, deallocating any resources claimed by its tasks. This is more efficient from a resource usage standpoint than pausing the connector, but can cause it to take longer to begin processing data once resumed. Note that the offsets for a connector can be only modified via the offsets management endpoints if it is in the stopped state + * `PUT /connectors/{name}/resume` \- resume a paused or stopped connector (or do nothing if the connector is not paused or stopped) + * `POST /connectors/{name}/restart?includeTasks=&onlyFailed=` \- restart a connector and its tasks instances. + * the "includeTasks" parameter specifies whether to restart the connector instance and task instances ("includeTasks=true") or just the connector instance ("includeTasks=false"), with the default ("false") preserving the same behavior as earlier versions. + * the "onlyFailed" parameter specifies whether to restart just the instances with a FAILED status ("onlyFailed=true") or all instances ("onlyFailed=false"), with the default ("false") preserving the same behavior as earlier versions. + * `POST /connectors/{name}/tasks/{taskId}/restart` \- restart an individual task (typically because it has failed) + * `DELETE /connectors/{name}` \- delete a connector, halting all tasks and deleting its configuration + * `GET /connectors/{name}/topics` \- get the set of topics that a specific connector is using since the connector was created or since a request to reset its set of active topics was issued + * `PUT /connectors/{name}/topics/reset` \- send a request to empty the set of active topics of a connector + * Offsets management endpoints (see [KIP-875](https://cwiki.apache.org/confluence/display/KAFKA/KIP-875%3A+First-class+offsets+support+in+Kafka+Connect) for more details): + * `GET /connectors/{name}/offsets` \- get the current offsets for a connector + * `DELETE /connectors/{name}/offsets` \- reset the offsets for a connector. The connector must exist and must be in the stopped state (see `PUT /connectors/{name}/stop`) + * `PATCH /connectors/{name}/offsets` \- alter the offsets for a connector. The connector must exist and must be in the stopped state (see `PUT /connectors/{name}/stop`). The request body should be a JSON object containing a JSON array `offsets` field, similar to the response body of the `GET /connectors/{name}/offsets` endpoint. An example request body for the `FileStreamSourceConnector`: + + { + "offsets": [ + { + "partition": { + "filename": "test.txt" + }, + "offset": { + "position": 30 + } + } + ] + } + + +An example request body for the `FileStreamSinkConnector`: + + { + "offsets": [ + { + "partition": { + "kafka_topic": "test", + "kafka_partition": 0 + }, + "offset": { + "kafka_offset": 5 + } + }, + { + "partition": { + "kafka_topic": "test", + "kafka_partition": 1 + }, + "offset": null + } + ] + } + + +The "offset" field may be null to reset the offset for a specific partition (applicable to both source and sink connectors). Note that the request body format depends on the connector implementation in the case of source connectors, whereas there is a common format across all sink connectors. + + + +Kafka Connect also provides a REST API for getting information about connector plugins: + + * `GET /connector-plugins`\- return a list of connector plugins installed in the Kafka Connect cluster. Note that the API only checks for connectors on the worker that handles the request, which means you may see inconsistent results, especially during a rolling upgrade if you add new connector jars + * `GET /connector-plugins/{plugin-type}/config` \- get the configuration definition for the specified plugin. + * `PUT /connector-plugins/{connector-type}/config/validate` \- validate the provided configuration values against the configuration definition. This API performs per config validation, returns suggested values and error messages during validation. + + + +The following is a supported REST request at the top-level (root) endpoint: + + * `GET /`\- return basic information about the Kafka Connect cluster such as the version of the Connect worker that serves the REST request (including git commit ID of the source code) and the Kafka cluster ID that is connected to. + + +The `admin.listeners` configuration can be used to configure admin REST APIs on Kafka Connect's REST API server. Similar to the `listeners` configuration, this field should contain a list of listeners in the following format: `protocol://host:port,protocol2://host2:port2`. Currently supported protocols are `http` and `https`. For example: + + + admin.listeners=http://localhost:8080,https://localhost:8443 + +By default, if `admin.listeners` is not configured, the admin REST APIs will be available on the regular listeners. + +The following are the currently supported admin REST API endpoints: + + * `GET /admin/loggers` \- list the current loggers that have their levels explicitly set and their log levels + * `GET /admin/loggers/{name}` \- get the log level for the specified logger + * `PUT /admin/loggers/{name}` \- set the log level for the specified logger + + + +See [KIP-495](https://cwiki.apache.org/confluence/display/KAFKA/KIP-495%3A+Dynamically+Adjust+Log+Levels+in+Connect) for more details about the admin logger REST APIs. + +For the complete specification of the Kafka Connect REST API, see the [OpenAPI documentation](/37/generated/connect_rest.yaml) + +## Error Reporting in Connect + +Kafka Connect provides error reporting to handle errors encountered along various stages of processing. By default, any error encountered during conversion or within transformations will cause the connector to fail. Each connector configuration can also enable tolerating such errors by skipping them, optionally writing each error and the details of the failed operation and problematic record (with various levels of detail) to the Connect application log. These mechanisms also capture errors when a sink connector is processing the messages consumed from its Kafka topics, and all of the errors can be written to a configurable "dead letter queue" (DLQ) Kafka topic. + +To report errors within a connector's converter, transforms, or within the sink connector itself to the log, set `errors.log.enable=true` in the connector configuration to log details of each error and problem record's topic, partition, and offset. For additional debugging purposes, set `errors.log.include.messages=true` to also log the problem record key, value, and headers to the log (note this may log sensitive information). + +To report errors within a connector's converter, transforms, or within the sink connector itself to a dead letter queue topic, set `errors.deadletterqueue.topic.name`, and optionally `errors.deadletterqueue.context.headers.enable=true`. + +By default connectors exhibit "fail fast" behavior immediately upon an error or exception. This is equivalent to adding the following configuration properties with their defaults to a connector configuration: + + + # disable retries on failure + errors.retry.timeout=0 + + # do not log the error and their contexts + errors.log.enable=false + + # do not record errors in a dead letter queue topic + errors.deadletterqueue.topic.name= + + # Fail on first error + errors.tolerance=none + +These and other related connector configuration properties can be changed to provide different behavior. For example, the following configuration properties can be added to a connector configuration to setup error handling with multiple retries, logging to the application logs and the `my-connector-errors` Kafka topic, and tolerating all errors by reporting them rather than failing the connector task: + + + # retry for at most 10 minutes times waiting up to 30 seconds between consecutive failures + errors.retry.timeout=600000 + errors.retry.delay.max.ms=30000 + + # log error context along with application logs, but do not include configs and messages + errors.log.enable=true + errors.log.include.messages=false + + # produce error context into the Kafka topic + errors.deadletterqueue.topic.name=my-connector-errors + + # Tolerate all errors. + errors.tolerance=all + +## Exactly-once support + +Kafka Connect is capable of providing exactly-once semantics for sink connectors (as of version 0.11.0) and source connectors (as of version 3.3.0). Please note that **support for exactly-once semantics is highly dependent on the type of connector you run.** Even if you set all the correct worker properties in the configuration for each node in a cluster, if a connector is not designed to, or cannot take advantage of the capabilities of the Kafka Connect framework, exactly-once may not be possible. + +### Sink connectors + +If a sink connector supports exactly-once semantics, to enable exactly-once at the Connect worker level, you must ensure its consumer group is configured to ignore records in aborted transactions. You can do this by setting the worker property `consumer.isolation.level` to `read_committed` or, if running a version of Kafka Connect that supports it, using a connector client config override policy that allows the `consumer.override.isolation.level` property to be set to `read_committed` in individual connector configs. There are no additional ACL requirements. + +### Source connectors + +If a source connector supports exactly-once semantics, you must configure your Connect cluster to enable framework-level support for exactly-once source connectors. Additional ACLs may be necessary if running against a secured Kafka cluster. Note that exactly-once support for source connectors is currently only available in distributed mode; standalone Connect workers cannot provide exactly-once semantics. + +#### Worker configuration + +For new Connect clusters, set the `exactly.once.source.support` property to `enabled` in the worker config for each node in the cluster. For existing clusters, two rolling upgrades are necessary. During the first upgrade, the `exactly.once.source.support` property should be set to `preparing`, and during the second, it should be set to `enabled`. + +#### ACL requirements + +With exactly-once source support enabled, or with `exactly.once.source.support` set to `preparing`, the principal for each Connect worker will require the following ACLs: + +Operation | Resource Type | Resource Name | Note +---|---|---|--- +Write | TransactionalId | `connect-cluster-${groupId}`, where `${groupId}` is the `group.id` of the cluster | +Describe | TransactionalId | `connect-cluster-${groupId}`, where `${groupId}` is the `group.id` of the cluster | +IdempotentWrite | Cluster | ID of the Kafka cluster that hosts the worker's config topic | The IdempotentWrite ACL has been deprecated as of 2.8 and will only be necessary for Connect clusters running on pre-2.8 Kafka clusters + +And with exactly-once source enabled (but not if `exactly.once.source.support` is set to `preparing`), the principal for each individual connector will require the following ACLs: + +Operation | Resource Type | Resource Name | Note +---|---|---|--- +Write | TransactionalId | `${groupId}-${connector}-${taskId}`, for each task that the connector will create, where `${groupId}` is the `group.id` of the Connect cluster, `${connector}` is the name of the connector, and `${taskId}` is the ID of the task (starting from zero) | A wildcard prefix of `${groupId}-${connector}*` can be used for convenience if there is no risk of conflict with other transactional IDs or if conflicts are acceptable to the user. +Describe | TransactionalId | `${groupId}-${connector}-${taskId}`, for each task that the connector will create, where `${groupId}` is the `group.id` of the Connect cluster, `${connector}` is the name of the connector, and `${taskId}` is the ID of the task (starting from zero) | A wildcard prefix of `${groupId}-${connector}*` can be used for convenience if there is no risk of conflict with other transactional IDs or if conflicts are acceptable to the user. +Write | Topic | Offsets topic used by the connector, which is either the value of the `offsets.storage.topic` property in the connector’s configuration if provided, or the value of the `offsets.storage.topic` property in the worker’s configuration if not. | +Read | Topic | Offsets topic used by the connector, which is either the value of the `offsets.storage.topic` property in the connector’s configuration if provided, or the value of the `offsets.storage.topic` property in the worker’s configuration if not. | +Describe | Topic | Offsets topic used by the connector, which is either the value of the `offsets.storage.topic` property in the connector’s configuration if provided, or the value of the `offsets.storage.topic` property in the worker’s configuration if not. | +Create | Topic | Offsets topic used by the connector, which is either the value of the `offsets.storage.topic` property in the connector’s configuration if provided, or the value of the `offsets.storage.topic` property in the worker’s configuration if not. | Only necessary if the offsets topic for the connector does not exist yet +IdempotentWrite | Cluster | ID of the Kafka cluster that the source connector writes to | The IdempotentWrite ACL has been deprecated as of 2.8 and will only be necessary for Connect clusters running on pre-2.8 Kafka clusters + +## Plugin Discovery + +Plugin discovery is the name for the strategy which the Connect worker uses to find plugin classes and make them accessible to configure and run in connectors. This is controlled by the plugin.discovery worker configuration, and has a significant impact on worker startup time. `service_load` is the fastest strategy, but care should be taken to verify that plugins are compatible before setting this configuration to `service_load`. + +Prior to version 3.6, this strategy was not configurable, and behaved like the `only_scan` mode which is compatible with all plugins. For version 3.6 and later, this mode defaults to `hybrid_warn` which is also compatible with all plugins, but logs a warning for plugins which are incompatible with `service_load`. The `hybrid_fail` strategy stops the worker with an error if a plugin incompatible with `service_load` is detected, asserting that all plugins are compatible. Finally, the `service_load` strategy disables the slow legacy scanning mechanism used in all other modes, and instead uses the faster `ServiceLoader` mechanism. Plugins which are incompatible with that mechanism may be unusable. + +### Verifying Plugin Compatibility + +To verify if all of your plugins are compatible with `service_load`, first ensure that you are using version 3.6 or later of Kafka Connect. You can then perform one of the following checks: + + * Start your worker with the default `hybrid_warn`strategy, and WARN logs enabled for the `org.apache.kafka.connect` package. At least one WARN log message mentioning the `plugin.discovery` configuration should be printed. This log message will explicitly say that all plugins are compatible, or list the incompatible plugins. + * Start your worker in a test environment with `hybrid_fail`. If all plugins are compatible, startup will succeed. If at least one plugin is not compatible the worker will fail to start up, and all incompatible plugins will be listed in the exception. + + + +If the verification step succeeds, then your current set of installed plugins is compatible, and it should be safe to change the `plugin.discovery` configuration to `service_load`. If the verification fails, you cannot use `service_load` strategy and should take note of the list of incompatible plugins. All plugins must be addressed before using the `service_load` strategy. It is recommended to perform this verification after installing or changing plugin versions, and the verification can be done automatically in a Continuous Integration environment. + +### Operators: Artifact Migration + +As an operator of Connect, if you discover incompatible plugins, there are multiple ways to resolve the incompatibility. They are listed below from most to least preferable. + + 1. Check the latest release from your plugin provider, and if it is compatible, upgrade. + 2. Contact your plugin provider and request that they migrate the plugin to be compatible, following the source migration instructions, and then upgrade to the compatible version. + 3. Migrate the plugin artifacts yourself using the included migration script. + + + +The migration script is located in `bin/connect-plugin-path.sh` and `bin\windows\connect-plugin-path.bat` of your Kafka installation. The script can migrate incompatible plugin artifacts already installed on your Connect worker's `plugin.path` by adding or modifying JAR or resource files. This is not suitable for environments using code-signing, as this can change artifacts such that they will fail signature verification. View the built-in help with `--help`. + +To perform a migration, first use the `list` subcommand to get an overview of the plugins available to the script. You must tell the script where to find plugins, which can be done with the repeatable `--worker-config`, `--plugin-path`, and `--plugin-location` arguments. The script will ignore plugins on the classpath, so any custom plugins on your classpath should be moved to the plugin path in order to be used with this migration script, or migrated manually. Be sure to compare the output of `list` with the worker startup warning or error message to ensure that all of your affected plugins are found by the script. + +Once you see that all incompatible plugins are included in the listing, you can proceed to dry-run the migration with `sync-manifests --dry-run`. This will perform all parts of the migration, except for writing the results of the migration to disk. Note that the `sync-manifests` command requires all specified paths to be writable, and may alter the contents of the directories. Make a backup of your plugins in the specified paths, or copy them to a writable directory. + +Ensure that you have a backup of your plugins and the dry-run succeeds before removing the `--dry-run` flag and actually running the migration. If the migration fails without the `--dry-run` flag, then the partially migrated artifacts should be discarded. The migration is idempotent, so running it multiple times and on already-migrated plugins is safe. After the script finishes, you should verify the migration is complete. The migration script is suitable for use in a Continuous Integration environment for automatic migration. + +### Developers: Source Migration + +To make plugins compatible with `service_load`, it is necessary to add [ServiceLoader](https://docs.oracle.com/javase/8/docs/api/java/util/ServiceLoader.html) manifests to your source code, which should then be packaged in the release artifact. Manifests are resource files in `META-INF/services/` named after their superclass type, and contain a list of fully-qualified subclass names, one on each line. + +In order for a plugin to be compatible, it must appear as a line in a manifest corresponding to the plugin superclass it extends. If a single plugin implements multiple plugin interfaces, then it should appear in a manifest for each interface it implements. If you have no classes for a certain type of plugin, you do not need to include a manifest file for that type. If you have classes which should not be visible as plugins, they should be marked abstract. The following types are expected to have manifests: + + * `org.apache.kafka.connect.sink.SinkConnector` + * `org.apache.kafka.connect.source.SourceConnector` + * `org.apache.kafka.connect.storage.Converter` + * `org.apache.kafka.connect.storage.HeaderConverter` + * `org.apache.kafka.connect.transforms.Transformation` + * `org.apache.kafka.connect.transforms.predicates.Predicate` + * `org.apache.kafka.common.config.provider.ConfigProvider` + * `org.apache.kafka.connect.rest.ConnectRestExtension` + * `org.apache.kafka.connect.connector.policy.ConnectorClientConfigOverridePolicy` + + + +For example, if you only have one connector with the fully-qualified name `com.example.MySinkConnector`, then only one manifest file must be added to resources in `META-INF/services/org.apache.kafka.connect.sink.SinkConnector`, and the contents should be similar to the following: + + + # license header or comment + com.example.MySinkConnector + +You should then verify that your manifests are correct by using the verification steps with a pre-release artifact. If the verification succeeds, you can then release the plugin normally, and operators can upgrade to the compatible version. diff --git a/docs/migration.html b/docs/migration.html deleted file mode 100644 index 95fc87ffacafe..0000000000000 --- a/docs/migration.html +++ /dev/null @@ -1,34 +0,0 @@ - - - -

Migrating from 0.7.x to 0.8

- -0.8 is our first (and hopefully last) release with a non-backwards-compatible wire protocol, ZooKeeper layout, and on-disk data format. This was a chance for us to clean up a lot of cruft and start fresh. This means performing a no-downtime upgrade is more painful than normal—you cannot just swap in the new code in-place. - -

Migration Steps

- -
    -
  1. Setup a new cluster running 0.8. -
  2. Use the 0.7 to 0.8 migration tool to mirror data from the 0.7 cluster into the 0.8 cluster. -
  3. When the 0.8 cluster is fully caught up, redeploy all data consumers running the 0.8 client and reading from the 0.8 cluster. -
  4. Finally migrate all 0.7 producers to 0.8 client publishing data to the 0.8 cluster. -
  5. Decommission the 0.7 cluster. -
  6. Drink. -
- - diff --git a/docs/operations/_index.md b/docs/operations/_index.md new file mode 100644 index 0000000000000..1ae5b53f63ef1 --- /dev/null +++ b/docs/operations/_index.md @@ -0,0 +1,10 @@ +--- +title: Operations +description: +weight: 6 +tags: ['kafka', 'docs', 'ops'] +aliases: +keywords: +type: docs +--- + diff --git a/docs/operations/basic-kafka-operations.md b/docs/operations/basic-kafka-operations.md new file mode 100644 index 0000000000000..73ee2ee55f06d --- /dev/null +++ b/docs/operations/basic-kafka-operations.md @@ -0,0 +1,552 @@ +--- +title: Basic Kafka Operations +description: Basic Kafka Operations +weight: 1 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +This section will review the most common operations you will perform on your Kafka cluster. All of the tools reviewed in this section are available under the `bin/` directory of the Kafka distribution and each tool will print details on all possible commandline options if it is run with no arguments. + +## Adding and removing topics + +You have the option of either adding topics manually or having them be created automatically when data is first published to a non-existent topic. If topics are auto-created then you may want to tune the default topic configurations used for auto-created topics. + +Topics are added and modified using the topic tool: + + + > bin/kafka-topics.sh --bootstrap-server broker_host:port --create --topic my_topic_name \ + --partitions 20 --replication-factor 3 --config x=y + +The replication factor controls how many servers will replicate each message that is written. If you have a replication factor of 3 then up to 2 servers can fail before you will lose access to your data. We recommend you use a replication factor of 2 or 3 so that you can transparently bounce machines without interrupting data consumption. + +The partition count controls how many logs the topic will be sharded into. There are several impacts of the partition count. First each partition must fit entirely on a single server. So if you have 20 partitions the full data set (and read and write load) will be handled by no more than 20 servers (not counting replicas). Finally the partition count impacts the maximum parallelism of your consumers. This is discussed in greater detail in the concepts section. + +Each sharded partition log is placed into its own folder under the Kafka log directory. The name of such folders consists of the topic name, appended by a dash (-) and the partition id. Since a typical folder name can not be over 255 characters long, there will be a limitation on the length of topic names. We assume the number of partitions will not ever be above 100,000. Therefore, topic names cannot be longer than 249 characters. This leaves just enough room in the folder name for a dash and a potentially 5 digit long partition id. + +The configurations added on the command line override the default settings the server has for things like the length of time data should be retained. The complete set of per-topic configurations is documented here. + +## Modifying topics + +You can change the configuration or partitioning of a topic using the same topic tool. + +To add partitions you can do + + + > bin/kafka-topics.sh --bootstrap-server broker_host:port --alter --topic my_topic_name \ + --partitions 40 + +Be aware that one use case for partitions is to semantically partition data, and adding partitions doesn't change the partitioning of existing data so this may disturb consumers if they rely on that partition. That is if data is partitioned by `hash(key) % number_of_partitions` then this partitioning will potentially be shuffled by adding partitions but Kafka will not attempt to automatically redistribute data in any way. + +To add configs: + + + > bin/kafka-configs.sh --bootstrap-server broker_host:port --entity-type topics --entity-name my_topic_name --alter --add-config x=y + +To remove a config: + + + > bin/kafka-configs.sh --bootstrap-server broker_host:port --entity-type topics --entity-name my_topic_name --alter --delete-config x + +And finally deleting a topic: + + + > bin/kafka-topics.sh --bootstrap-server broker_host:port --delete --topic my_topic_name + +Kafka does not currently support reducing the number of partitions for a topic. + +Instructions for changing the replication factor of a topic can be found here. + +## Graceful shutdown + +The Kafka cluster will automatically detect any broker shutdown or failure and elect new leaders for the partitions on that machine. This will occur whether a server fails or it is brought down intentionally for maintenance or configuration changes. For the latter cases Kafka supports a more graceful mechanism for stopping a server than just killing it. When a server is stopped gracefully it has two optimizations it will take advantage of: + + 1. It will sync all its logs to disk to avoid needing to do any log recovery when it restarts (i.e. validating the checksum for all messages in the tail of the log). Log recovery takes time so this speeds up intentional restarts. + 2. It will migrate any partitions the server is the leader for to other replicas prior to shutting down. This will make the leadership transfer faster and minimize the time each partition is unavailable to a few milliseconds. +Syncing the logs will happen automatically whenever the server is stopped other than by a hard kill, but the controlled leadership migration requires using a special setting: + + + controlled.shutdown.enable=true + +Note that controlled shutdown will only succeed if _all_ the partitions hosted on the broker have replicas (i.e. the replication factor is greater than 1 _and_ at least one of these replicas is alive). This is generally what you want since shutting down the last replica would make that topic partition unavailable. + +## Balancing leadership + +Whenever a broker stops or crashes, leadership for that broker's partitions transfers to other replicas. When the broker is restarted it will only be a follower for all its partitions, meaning it will not be used for client reads and writes. + +To avoid this imbalance, Kafka has a notion of preferred replicas. If the list of replicas for a partition is 1,5,9 then node 1 is preferred as the leader to either node 5 or 9 because it is earlier in the replica list. By default the Kafka cluster will try to restore leadership to the preferred replicas. This behaviour is configured with: + + + auto.leader.rebalance.enable=true + +You can also set this to false, but you will then need to manually restore leadership to the restored replicas by running the command: + + + > bin/kafka-leader-election.sh --bootstrap-server broker_host:port --election-type preferred --all-topic-partitions + +## Balancing Replicas Across Racks + +The rack awareness feature spreads replicas of the same partition across different racks. This extends the guarantees Kafka provides for broker-failure to cover rack-failure, limiting the risk of data loss should all the brokers on a rack fail at once. The feature can also be applied to other broker groupings such as availability zones in EC2. + +You can specify that a broker belongs to a particular rack by adding a property to the broker config: + + + broker.rack=my-rack-id + +When a topic is created, modified or replicas are redistributed, the rack constraint will be honoured, ensuring replicas span as many racks as they can (a partition will span min(#racks, replication-factor) different racks). + +The algorithm used to assign replicas to brokers ensures that the number of leaders per broker will be constant, regardless of how brokers are distributed across racks. This ensures balanced throughput. + +However if racks are assigned different numbers of brokers, the assignment of replicas will not be even. Racks with fewer brokers will get more replicas, meaning they will use more storage and put more resources into replication. Hence it is sensible to configure an equal number of brokers per rack. + +## Mirroring data between clusters & Geo-replication + +Kafka administrators can define data flows that cross the boundaries of individual Kafka clusters, data centers, or geographical regions. Please refer to the section on Geo-Replication for further information. + +## Checking consumer position + +Sometimes it's useful to see the position of your consumers. We have a tool that will show the position of all consumers in a consumer group as well as how far behind the end of the log they are. To run this tool on a consumer group named _my-group_ consuming a topic named _my-topic_ would look like this: + + + > bin/kafka-consumer-groups.sh --bootstrap-server localhost:9092 --describe --group my-group + + TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID HOST CLIENT-ID + my-topic 0 2 4 2 consumer-1-029af89c-873c-4751-a720-cefd41a669d6 /127.0.0.1 consumer-1 + my-topic 1 2 3 1 consumer-1-029af89c-873c-4751-a720-cefd41a669d6 /127.0.0.1 consumer-1 + my-topic 2 2 3 1 consumer-2-42c1abd4-e3b2-425d-a8bb-e1ea49b29bb2 /127.0.0.1 consumer-2 + +## Managing Consumer Groups + +With the ConsumerGroupCommand tool, we can list, describe, or delete the consumer groups. The consumer group can be deleted manually, or automatically when the last committed offset for that group expires. Manual deletion works only if the group does not have any active members. For example, to list all consumer groups across all topics: + + + > bin/kafka-consumer-groups.sh --bootstrap-server localhost:9092 --list + + test-consumer-group + +To view offsets, as mentioned earlier, we "describe" the consumer group like this: + + + > bin/kafka-consumer-groups.sh --bootstrap-server localhost:9092 --describe --group my-group + + TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID HOST CLIENT-ID + topic3 0 241019 395308 154289 consumer2-e76ea8c3-5d30-4299-9005-47eb41f3d3c4 /127.0.0.1 consumer2 + topic2 1 520678 803288 282610 consumer2-e76ea8c3-5d30-4299-9005-47eb41f3d3c4 /127.0.0.1 consumer2 + topic3 1 241018 398817 157799 consumer2-e76ea8c3-5d30-4299-9005-47eb41f3d3c4 /127.0.0.1 consumer2 + topic1 0 854144 855809 1665 consumer1-3fc8d6f1-581a-4472-bdf3-3515b4aee8c1 /127.0.0.1 consumer1 + topic2 0 460537 803290 342753 consumer1-3fc8d6f1-581a-4472-bdf3-3515b4aee8c1 /127.0.0.1 consumer1 + topic3 2 243655 398812 155157 consumer4-117fe4d3-c6c1-4178-8ee9-eb4a3954bee0 /127.0.0.1 consumer4 + +There are a number of additional "describe" options that can be used to provide more detailed information about a consumer group: + + * \--members: This option provides the list of all active members in the consumer group. + + > bin/kafka-consumer-groups.sh --bootstrap-server localhost:9092 --describe --group my-group --members + + CONSUMER-ID HOST CLIENT-ID #PARTITIONS + consumer1-3fc8d6f1-581a-4472-bdf3-3515b4aee8c1 /127.0.0.1 consumer1 2 + consumer4-117fe4d3-c6c1-4178-8ee9-eb4a3954bee0 /127.0.0.1 consumer4 1 + consumer2-e76ea8c3-5d30-4299-9005-47eb41f3d3c4 /127.0.0.1 consumer2 3 + consumer3-ecea43e4-1f01-479f-8349-f9130b75d8ee /127.0.0.1 consumer3 0 + + * \--members --verbose: On top of the information reported by the "--members" options above, this option also provides the partitions assigned to each member. + + > bin/kafka-consumer-groups.sh --bootstrap-server localhost:9092 --describe --group my-group --members --verbose + + CONSUMER-ID HOST CLIENT-ID #PARTITIONS ASSIGNMENT + consumer1-3fc8d6f1-581a-4472-bdf3-3515b4aee8c1 /127.0.0.1 consumer1 2 topic1(0), topic2(0) + consumer4-117fe4d3-c6c1-4178-8ee9-eb4a3954bee0 /127.0.0.1 consumer4 1 topic3(2) + consumer2-e76ea8c3-5d30-4299-9005-47eb41f3d3c4 /127.0.0.1 consumer2 3 topic2(1), topic3(0,1) + consumer3-ecea43e4-1f01-479f-8349-f9130b75d8ee /127.0.0.1 consumer3 0 - + + * \--offsets: This is the default describe option and provides the same output as the "--describe" option. + * \--state: This option provides useful group-level information. + + > bin/kafka-consumer-groups.sh --bootstrap-server localhost:9092 --describe --group my-group --state + + COORDINATOR (ID) ASSIGNMENT-STRATEGY STATE #MEMBERS + localhost:9092 (0) range Stable 4 + + +To manually delete one or multiple consumer groups, the "--delete" option can be used: + + + > bin/kafka-consumer-groups.sh --bootstrap-server localhost:9092 --delete --group my-group --group my-other-group + + Deletion of requested consumer groups ('my-group', 'my-other-group') was successful. + +To reset offsets of a consumer group, "--reset-offsets" option can be used. This option supports one consumer group at the time. It requires defining following scopes: --all-topics or --topic. One scope must be selected, unless you use '--from-file' scenario. Also, first make sure that the consumer instances are inactive. See [KIP-122](https://cwiki.apache.org/confluence/display/KAFKA/KIP-122%3A+Add+Reset+Consumer+Group+Offsets+tooling) for more details. + +It has 3 execution options: + + * (default) to display which offsets to reset. + * \--execute : to execute --reset-offsets process. + * \--export : to export the results to a CSV format. + + + +\--reset-offsets also has following scenarios to choose from (at least one scenario must be selected): + + * \--to-datetime : Reset offsets to offsets from datetime. Format: 'YYYY-MM-DDTHH:mm:SS.sss' + * \--to-earliest : Reset offsets to earliest offset. + * \--to-latest : Reset offsets to latest offset. + * \--shift-by : Reset offsets shifting current offset by 'n', where 'n' can be positive or negative. + * \--from-file : Reset offsets to values defined in CSV file. + * \--to-current : Resets offsets to current offset. + * \--by-duration : Reset offsets to offset by duration from current timestamp. Format: 'PnDTnHnMnS' + * \--to-offset : Reset offsets to a specific offset. + +Please note, that out of range offsets will be adjusted to available offset end. For example, if offset end is at 10 and offset shift request is of 15, then, offset at 10 will actually be selected. + +For example, to reset offsets of a consumer group to the latest offset: + + + > bin/kafka-consumer-groups.sh --bootstrap-server localhost:9092 --reset-offsets --group consumergroup1 --topic topic1 --to-latest + + TOPIC PARTITION NEW-OFFSET + topic1 0 0 + +If you are using the old high-level consumer and storing the group metadata in ZooKeeper (i.e. `offsets.storage=zookeeper`), pass `--zookeeper` instead of `--bootstrap-server`: + + + > bin/kafka-consumer-groups.sh --zookeeper localhost:2181 --list + +## Expanding your cluster + +Adding servers to a Kafka cluster is easy, just assign them a unique broker id and start up Kafka on your new servers. However these new servers will not automatically be assigned any data partitions, so unless partitions are moved to them they won't be doing any work until new topics are created. So usually when you add machines to your cluster you will want to migrate some existing data to these machines. + +The process of migrating data is manually initiated but fully automated. Under the covers what happens is that Kafka will add the new server as a follower of the partition it is migrating and allow it to fully replicate the existing data in that partition. When the new server has fully replicated the contents of this partition and joined the in-sync replica one of the existing replicas will delete their partition's data. + +The partition reassignment tool can be used to move partitions across brokers. An ideal partition distribution would ensure even data load and partition sizes across all brokers. The partition reassignment tool does not have the capability to automatically study the data distribution in a Kafka cluster and move partitions around to attain an even load distribution. As such, the admin has to figure out which topics or partitions should be moved around. + +The partition reassignment tool can run in 3 mutually exclusive modes: + + * \--generate: In this mode, given a list of topics and a list of brokers, the tool generates a candidate reassignment to move all partitions of the specified topics to the new brokers. This option merely provides a convenient way to generate a partition reassignment plan given a list of topics and target brokers. + * \--execute: In this mode, the tool kicks off the reassignment of partitions based on the user provided reassignment plan. (using the --reassignment-json-file option). This can either be a custom reassignment plan hand crafted by the admin or provided by using the --generate option + * \--verify: In this mode, the tool verifies the status of the reassignment for all partitions listed during the last --execute. The status can be either of successfully completed, failed or in progress + + + +### Automatically migrating data to new machines + +The partition reassignment tool can be used to move some topics off of the current set of brokers to the newly added brokers. This is typically useful while expanding an existing cluster since it is easier to move entire topics to the new set of brokers, than moving one partition at a time. When used to do this, the user should provide a list of topics that should be moved to the new set of brokers and a target list of new brokers. The tool then evenly distributes all partitions for the given list of topics across the new set of brokers. During this move, the replication factor of the topic is kept constant. Effectively the replicas for all partitions for the input list of topics are moved from the old set of brokers to the newly added brokers. + +For instance, the following example will move all partitions for topics foo1,foo2 to the new set of brokers 5,6. At the end of this move, all partitions for topics foo1 and foo2 will _only_ exist on brokers 5,6. + +Since the tool accepts the input list of topics as a json file, you first need to identify the topics you want to move and create the json file as follows: + + + > cat topics-to-move.json + {"topics": [{"topic": "foo1"}, + {"topic": "foo2"}], + "version":1 + } + +Once the json file is ready, use the partition reassignment tool to generate a candidate assignment: + + + > bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --topics-to-move-json-file topics-to-move.json --broker-list "5,6" --generate + Current partition replica assignment + + {"version":1, + "partitions":[{"topic":"foo1","partition":0,"replicas":[2,1]}, + {"topic":"foo1","partition":1,"replicas":[1,3]}, + {"topic":"foo1","partition":2,"replicas":[3,4]}, + {"topic":"foo2","partition":0,"replicas":[4,2]}, + {"topic":"foo2","partition":1,"replicas":[2,1]}, + {"topic":"foo2","partition":2,"replicas":[1,3]}] + } + + Proposed partition reassignment configuration + + {"version":1, + "partitions":[{"topic":"foo1","partition":0,"replicas":[6,5]}, + {"topic":"foo1","partition":1,"replicas":[5,6]}, + {"topic":"foo1","partition":2,"replicas":[6,5]}, + {"topic":"foo2","partition":0,"replicas":[5,6]}, + {"topic":"foo2","partition":1,"replicas":[6,5]}, + {"topic":"foo2","partition":2,"replicas":[5,6]}] + } + +The tool generates a candidate assignment that will move all partitions from topics foo1,foo2 to brokers 5,6. Note, however, that at this point, the partition movement has not started, it merely tells you the current assignment and the proposed new assignment. The current assignment should be saved in case you want to rollback to it. The new assignment should be saved in a json file (e.g. expand-cluster-reassignment.json) to be input to the tool with the --execute option as follows: + + + > bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --reassignment-json-file expand-cluster-reassignment.json --execute + Current partition replica assignment + + {"version":1, + "partitions":[{"topic":"foo1","partition":0,"replicas":[2,1]}, + {"topic":"foo1","partition":1,"replicas":[1,3]}, + {"topic":"foo1","partition":2,"replicas":[3,4]}, + {"topic":"foo2","partition":0,"replicas":[4,2]}, + {"topic":"foo2","partition":1,"replicas":[2,1]}, + {"topic":"foo2","partition":2,"replicas":[1,3]}] + } + + Save this to use as the --reassignment-json-file option during rollback + Successfully started partition reassignments for foo1-0,foo1-1,foo1-2,foo2-0,foo2-1,foo2-2 + + +Finally, the --verify option can be used with the tool to check the status of the partition reassignment. Note that the same expand-cluster-reassignment.json (used with the --execute option) should be used with the --verify option: + + + > bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --reassignment-json-file expand-cluster-reassignment.json --verify + Status of partition reassignment: + Reassignment of partition [foo1,0] is completed + Reassignment of partition [foo1,1] is still in progress + Reassignment of partition [foo1,2] is still in progress + Reassignment of partition [foo2,0] is completed + Reassignment of partition [foo2,1] is completed + Reassignment of partition [foo2,2] is completed + +### Custom partition assignment and migration + +The partition reassignment tool can also be used to selectively move replicas of a partition to a specific set of brokers. When used in this manner, it is assumed that the user knows the reassignment plan and does not require the tool to generate a candidate reassignment, effectively skipping the --generate step and moving straight to the --execute step + +For instance, the following example moves partition 0 of topic foo1 to brokers 5,6 and partition 1 of topic foo2 to brokers 2,3: + +The first step is to hand craft the custom reassignment plan in a json file: + + + > cat custom-reassignment.json + {"version":1,"partitions":[{"topic":"foo1","partition":0,"replicas":[5,6]},{"topic":"foo2","partition":1,"replicas":[2,3]}]} + +Then, use the json file with the --execute option to start the reassignment process: + + + > bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --reassignment-json-file custom-reassignment.json --execute + Current partition replica assignment + + {"version":1, + "partitions":[{"topic":"foo1","partition":0,"replicas":[1,2]}, + {"topic":"foo2","partition":1,"replicas":[3,4]}] + } + + Save this to use as the --reassignment-json-file option during rollback + Successfully started partition reassignments for foo1-0,foo2-1 + + +The --verify option can be used with the tool to check the status of the partition reassignment. Note that the same custom-reassignment.json (used with the --execute option) should be used with the --verify option: + + + > bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --reassignment-json-file custom-reassignment.json --verify + Status of partition reassignment: + Reassignment of partition [foo1,0] is completed + Reassignment of partition [foo2,1] is completed + +## Decommissioning brokers + +The partition reassignment tool does not have the ability to automatically generate a reassignment plan for decommissioning brokers yet. As such, the admin has to come up with a reassignment plan to move the replica for all partitions hosted on the broker to be decommissioned, to the rest of the brokers. This can be relatively tedious as the reassignment needs to ensure that all the replicas are not moved from the decommissioned broker to only one other broker. To make this process effortless, we plan to add tooling support for decommissioning brokers in the future. + +## Increasing replication factor + +Increasing the replication factor of an existing partition is easy. Just specify the extra replicas in the custom reassignment json file and use it with the --execute option to increase the replication factor of the specified partitions. + +For instance, the following example increases the replication factor of partition 0 of topic foo from 1 to 3. Before increasing the replication factor, the partition's only replica existed on broker 5. As part of increasing the replication factor, we will add more replicas on brokers 6 and 7. + +The first step is to hand craft the custom reassignment plan in a json file: + + + > cat increase-replication-factor.json + {"version":1, + "partitions":[{"topic":"foo","partition":0,"replicas":[5,6,7]}]} + +Then, use the json file with the --execute option to start the reassignment process: + + + > bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --reassignment-json-file increase-replication-factor.json --execute + Current partition replica assignment + + {"version":1, + "partitions":[{"topic":"foo","partition":0,"replicas":[5]}]} + + Save this to use as the --reassignment-json-file option during rollback + Successfully started partition reassignment for foo-0 + +The --verify option can be used with the tool to check the status of the partition reassignment. Note that the same increase-replication-factor.json (used with the --execute option) should be used with the --verify option: + + + > bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --reassignment-json-file increase-replication-factor.json --verify + Status of partition reassignment: + Reassignment of partition [foo,0] is completed + +You can also verify the increase in replication factor with the kafka-topics tool: + + + > bin/kafka-topics.sh --bootstrap-server localhost:9092 --topic foo --describe + Topic:foo PartitionCount:1 ReplicationFactor:3 Configs: + Topic: foo Partition: 0 Leader: 5 Replicas: 5,6,7 Isr: 5,6,7 + +## Limiting Bandwidth Usage during Data Migration + +Kafka lets you apply a throttle to replication traffic, setting an upper bound on the bandwidth used to move replicas from machine to machine. This is useful when rebalancing a cluster, bootstrapping a new broker or adding or removing brokers, as it limits the impact these data-intensive operations will have on users. + +There are two interfaces that can be used to engage a throttle. The simplest, and safest, is to apply a throttle when invoking the kafka-reassign-partitions.sh, but kafka-configs.sh can also be used to view and alter the throttle values directly. + +So for example, if you were to execute a rebalance, with the below command, it would move partitions at no more than 50MB/s. + + + $ bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --execute --reassignment-json-file bigger-cluster.json --throttle 50000000 + +When you execute this script you will see the throttle engage: + + + The inter-broker throttle limit was set to 50000000 B/s + Successfully started partition reassignment for foo1-0 + +Should you wish to alter the throttle, during a rebalance, say to increase the throughput so it completes quicker, you can do this by re-running the execute command with the --additional option passing the same reassignment-json-file: + + + $ bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --additional --execute --reassignment-json-file bigger-cluster.json --throttle 700000000 + The inter-broker throttle limit was set to 700000000 B/s + +Once the rebalance completes the administrator can check the status of the rebalance using the --verify option. If the rebalance has completed, the throttle will be removed via the --verify command. It is important that administrators remove the throttle in a timely manner once rebalancing completes by running the command with the --verify option. Failure to do so could cause regular replication traffic to be throttled. + +When the --verify option is executed, and the reassignment has completed, the script will confirm that the throttle was removed: + + + > bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --verify --reassignment-json-file bigger-cluster.json + Status of partition reassignment: + Reassignment of partition [my-topic,1] is completed + Reassignment of partition [my-topic,0] is completed + + Clearing broker-level throttles on brokers 1,2,3 + Clearing topic-level throttles on topic my-topic + +The administrator can also validate the assigned configs using the kafka-configs.sh. There are two pairs of throttle configuration used to manage the throttling process. First pair refers to the throttle value itself. This is configured, at a broker level, using the dynamic properties: + + + leader.replication.throttled.rate + follower.replication.throttled.rate + +Then there is the configuration pair of enumerated sets of throttled replicas: + + + leader.replication.throttled.replicas + follower.replication.throttled.replicas + +Which are configured per topic. + +All four config values are automatically assigned by kafka-reassign-partitions.sh (discussed below). + +To view the throttle limit configuration: + + + > bin/kafka-configs.sh --describe --bootstrap-server localhost:9092 --entity-type brokers + Configs for brokers '2' are leader.replication.throttled.rate=700000000,follower.replication.throttled.rate=700000000 + Configs for brokers '1' are leader.replication.throttled.rate=700000000,follower.replication.throttled.rate=700000000 + +This shows the throttle applied to both leader and follower side of the replication protocol. By default both sides are assigned the same throttled throughput value. + +To view the list of throttled replicas: + + + > bin/kafka-configs.sh --describe --bootstrap-server localhost:9092 --entity-type topics + Configs for topic 'my-topic' are leader.replication.throttled.replicas=1:102,0:101, + follower.replication.throttled.replicas=1:101,0:102 + +Here we see the leader throttle is applied to partition 1 on broker 102 and partition 0 on broker 101. Likewise the follower throttle is applied to partition 1 on broker 101 and partition 0 on broker 102. + +By default kafka-reassign-partitions.sh will apply the leader throttle to all replicas that exist before the rebalance, any one of which might be leader. It will apply the follower throttle to all move destinations. So if there is a partition with replicas on brokers 101,102, being reassigned to 102,103, a leader throttle, for that partition, would be applied to 101,102 and a follower throttle would be applied to 103 only. + +If required, you can also use the --alter switch on kafka-configs.sh to alter the throttle configurations manually. + +### Safe usage of throttled replication + +Some care should be taken when using throttled replication. In particular: + +_(1) Throttle Removal:_ + +The throttle should be removed in a timely manner once reassignment completes (by running kafka-reassign-partitions.sh \--verify). + +_(2) Ensuring Progress:_ + +If the throttle is set too low, in comparison to the incoming write rate, it is possible for replication to not make progress. This occurs when: + + + max(BytesInPerSec) > throttle + +Where BytesInPerSec is the metric that monitors the write throughput of producers into each broker. + +The administrator can monitor whether replication is making progress, during the rebalance, using the metric: + + + kafka.server:type=FetcherLagMetrics,name=ConsumerLag,clientId=([-.\w]+),topic=([-.\w]+),partition=([0-9]+) + +The lag should constantly decrease during replication. If the metric does not decrease the administrator should increase the throttle throughput as described above. + +## Setting quotas + +Quotas overrides and defaults may be configured at (user, client-id), user or client-id levels as described here. By default, clients receive an unlimited quota. It is possible to set custom quotas for each (user, client-id), user or client-id group. + +Configure custom quota for (user=user1, client-id=clientA): + + + > bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --add-config 'producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200' --entity-type users --entity-name user1 --entity-type clients --entity-name clientA + Updated config for entity: user-principal 'user1', client-id 'clientA'. + +Configure custom quota for user=user1: + + + > bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --add-config 'producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200' --entity-type users --entity-name user1 + Updated config for entity: user-principal 'user1'. + +Configure custom quota for client-id=clientA: + + + > bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --add-config 'producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200' --entity-type clients --entity-name clientA + Updated config for entity: client-id 'clientA'. + +It is possible to set default quotas for each (user, client-id), user or client-id group by specifying _\--entity-default_ option instead of _\--entity-name_. + +Configure default client-id quota for user=userA: + + + > bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --add-config 'producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200' --entity-type users --entity-name user1 --entity-type clients --entity-default + Updated config for entity: user-principal 'user1', default client-id. + +Configure default quota for user: + + + > bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --add-config 'producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200' --entity-type users --entity-default + Updated config for entity: default user-principal. + +Configure default quota for client-id: + + + > bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --add-config 'producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200' --entity-type clients --entity-default + Updated config for entity: default client-id. + +Here's how to describe the quota for a given (user, client-id): + + + > bin/kafka-configs.sh --bootstrap-server localhost:9092 --describe --entity-type users --entity-name user1 --entity-type clients --entity-name clientA + Configs for user-principal 'user1', client-id 'clientA' are producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200 + +Describe quota for a given user: + + + > bin/kafka-configs.sh --bootstrap-server localhost:9092 --describe --entity-type users --entity-name user1 + Configs for user-principal 'user1' are producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200 + +Describe quota for a given client-id: + + + > bin/kafka-configs.sh --bootstrap-server localhost:9092 --describe --entity-type clients --entity-name clientA + Configs for client-id 'clientA' are producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200 + +If entity name is not specified, all entities of the specified type are described. For example, describe all users: + + + > bin/kafka-configs.sh --bootstrap-server localhost:9092 --describe --entity-type users + Configs for user-principal 'user1' are producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200 + Configs for default user-principal are producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200 + +Similarly for (user, client): + + + > bin/kafka-configs.sh --bootstrap-server localhost:9092 --describe --entity-type users --entity-type clients + Configs for user-principal 'user1', default client-id are producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200 + Configs for user-principal 'user1', client-id 'clientA' are producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200 diff --git a/docs/operations/datacenters.md b/docs/operations/datacenters.md new file mode 100644 index 0000000000000..c2b65ee9a7cb6 --- /dev/null +++ b/docs/operations/datacenters.md @@ -0,0 +1,21 @@ +--- +title: Datacenters +description: Datacenters +weight: 2 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +Some deployments will need to manage a data pipeline that spans multiple datacenters. Our recommended approach to this is to deploy a local Kafka cluster in each datacenter, with application instances in each datacenter interacting only with their local cluster and mirroring data between clusters (see the documentation on Geo-Replication for how to do this). + +This deployment pattern allows datacenters to act as independent entities and allows us to manage and tune inter-datacenter replication centrally. This allows each facility to stand alone and operate even if the inter-datacenter links are unavailable: when this occurs the mirroring falls behind until the link is restored at which time it catches up. + +For applications that need a global view of all data you can use mirroring to provide clusters which have aggregate data mirrored from the local clusters in _all_ datacenters. These aggregate clusters are used for reads by applications that require the full data set. + +This is not the only possible deployment pattern. It is possible to read from or write to a remote Kafka cluster over the WAN, though obviously this will add whatever latency is required to get the cluster. + +Kafka naturally batches data in both the producer and consumer so it can achieve high-throughput even over a high-latency connection. To allow this though it may be necessary to increase the TCP socket buffer sizes for the producer, consumer, and broker using the `socket.send.buffer.bytes` and `socket.receive.buffer.bytes` configurations. The appropriate way to set this is documented [here](https://en.wikipedia.org/wiki/Bandwidth-delay_product). + +It is generally _not_ advisable to run a _single_ Kafka cluster that spans multiple datacenters over a high-latency link. This will incur very high replication latency both for Kafka writes and ZooKeeper writes, and neither Kafka nor ZooKeeper will remain available in all locations if the network between locations is unavailable. diff --git a/docs/operations/geo-replication-(cross-cluster-data-mirroring).md b/docs/operations/geo-replication-(cross-cluster-data-mirroring).md new file mode 100644 index 0000000000000..4e3a65ec31570 --- /dev/null +++ b/docs/operations/geo-replication-(cross-cluster-data-mirroring).md @@ -0,0 +1,470 @@ +--- +title: Geo-Replication (Cross-Cluster Data Mirroring) +description: Geo-Replication (Cross-Cluster Data Mirroring) +weight: 3 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +## Geo-Replication Overview + +Kafka administrators can define data flows that cross the boundaries of individual Kafka clusters, data centers, or geo-regions. Such event streaming setups are often needed for organizational, technical, or legal requirements. Common scenarios include: + + * Geo-replication + * Disaster recovery + * Feeding edge clusters into a central, aggregate cluster + * Physical isolation of clusters (such as production vs. testing) + * Cloud migration or hybrid cloud deployments + * Legal and compliance requirements + + + +Administrators can set up such inter-cluster data flows with Kafka's MirrorMaker (version 2), a tool to replicate data between different Kafka environments in a streaming manner. MirrorMaker is built on top of the Kafka Connect framework and supports features such as: + + * Replicates topics (data plus configurations) + * Replicates consumer groups including offsets to migrate applications between clusters + * Replicates ACLs + * Preserves partitioning + * Automatically detects new topics and partitions + * Provides a wide range of metrics, such as end-to-end replication latency across multiple data centers/clusters + * Fault-tolerant and horizontally scalable operations + + + +_Note: Geo-replication with MirrorMaker replicates data across Kafka clusters. This inter-cluster replication is different from Kafka'sintra-cluster replication, which replicates data within the same Kafka cluster._ + +## What Are Replication Flows + +With MirrorMaker, Kafka administrators can replicate topics, topic configurations, consumer groups and their offsets, and ACLs from one or more source Kafka clusters to one or more target Kafka clusters, i.e., across cluster environments. In a nutshell, MirrorMaker uses Connectors to consume from source clusters and produce to target clusters. + +These directional flows from source to target clusters are called replication flows. They are defined with the format `{source_cluster}->{target_cluster}` in the MirrorMaker configuration file as described later. Administrators can create complex replication topologies based on these flows. + +Here are some example patterns: + + * Active/Active high availability deployments: `A->B, B->A` + * Active/Passive or Active/Standby high availability deployments: `A->B` + * Aggregation (e.g., from many clusters to one): `A->K, B->K, C->K` + * Fan-out (e.g., from one to many clusters): `K->A, K->B, K->C` + * Forwarding: `A->B, B->C, C->D` + + + +By default, a flow replicates all topics and consumer groups (except excluded ones). However, each replication flow can be configured independently. For instance, you can define that only specific topics or consumer groups are replicated from the source cluster to the target cluster. + +Here is a first example on how to configure data replication from a `primary` cluster to a `secondary` cluster (an active/passive setup): + + + # Basic settings + clusters = primary, secondary + primary.bootstrap.servers = broker3-primary:9092 + secondary.bootstrap.servers = broker5-secondary:9092 + + # Define replication flows + primary->secondary.enabled = true + primary->secondary.topics = foobar-topic, quux-.* + + +## Configuring Geo-Replication + +The following sections describe how to configure and run a dedicated MirrorMaker cluster. If you want to run MirrorMaker within an existing Kafka Connect cluster or other supported deployment setups, please refer to [KIP-382: MirrorMaker 2.0](https://cwiki.apache.org/confluence/display/KAFKA/KIP-382%3A+MirrorMaker+2.0) and be aware that the names of configuration settings may vary between deployment modes. + +Beyond what's covered in the following sections, further examples and information on configuration settings are available at: + + * [MirrorMakerConfig](https://github.com/apache/kafka/blob/trunk/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java), [MirrorConnectorConfig](https://github.com/apache/kafka/blob/trunk/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorConnectorConfig.java) + * [DefaultTopicFilter](https://github.com/apache/kafka/blob/trunk/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/DefaultTopicFilter.java) for topics, [DefaultGroupFilter](https://github.com/apache/kafka/blob/trunk/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/DefaultGroupFilter.java) for consumer groups + * Example configuration settings in [connect-mirror-maker.properties](https://github.com/apache/kafka/blob/trunk/config/connect-mirror-maker.properties), [KIP-382: MirrorMaker 2.0](https://cwiki.apache.org/confluence/display/KAFKA/KIP-382%3A+MirrorMaker+2.0) + + + +### Configuration File Syntax + +The MirrorMaker configuration file is typically named `connect-mirror-maker.properties`. You can configure a variety of components in this file: + + * MirrorMaker settings: global settings including cluster definitions (aliases), plus custom settings per replication flow + * Kafka Connect and connector settings + * Kafka producer, consumer, and admin client settings + + + +Example: Define MirrorMaker settings (explained in more detail later). + + + # Global settings + clusters = us-west, us-east # defines cluster aliases + us-west.bootstrap.servers = broker3-west:9092 + us-east.bootstrap.servers = broker5-east:9092 + + topics = .* # all topics to be replicated by default + + # Specific replication flow settings (here: flow from us-west to us-east) + us-west->us-east.enabled = true + us-west->us.east.topics = foo.*, bar.* # override the default above + + +MirrorMaker is based on the Kafka Connect framework. Any Kafka Connect, source connector, and sink connector settings as described in the documentation chapter on Kafka Connect can be used directly in the MirrorMaker configuration, without having to change or prefix the name of the configuration setting. + +Example: Define custom Kafka Connect settings to be used by MirrorMaker. + + + # Setting Kafka Connect defaults for MirrorMaker + tasks.max = 5 + + +Most of the default Kafka Connect settings work well for MirrorMaker out-of-the-box, with the exception of `tasks.max`. In order to evenly distribute the workload across more than one MirrorMaker process, it is recommended to set `tasks.max` to at least `2` (preferably higher) depending on the available hardware resources and the total number of topic-partitions to be replicated. + +You can further customize MirrorMaker's Kafka Connect settings _per source or target cluster_ (more precisely, you can specify Kafka Connect worker-level configuration settings "per connector"). Use the format of `{cluster}.{config_name}` in the MirrorMaker configuration file. + +Example: Define custom connector settings for the `us-west` cluster. + + + # us-west custom settings + us-west.offset.storage.topic = my-mirrormaker-offsets + + +MirrorMaker internally uses the Kafka producer, consumer, and admin clients. Custom settings for these clients are often needed. To override the defaults, use the following format in the MirrorMaker configuration file: + + * `{source}.consumer.{consumer_config_name}` + * `{target}.producer.{producer_config_name}` + * `{source_or_target}.admin.{admin_config_name}` + + + +Example: Define custom producer, consumer, admin client settings. + + + # us-west cluster (from which to consume) + us-west.consumer.isolation.level = read_committed + us-west.admin.bootstrap.servers = broker57-primary:9092 + + # us-east cluster (to which to produce) + us-east.producer.compression.type = gzip + us-east.producer.buffer.memory = 32768 + us-east.admin.bootstrap.servers = broker8-secondary:9092 + + +### Exactly once + +Exactly-once semantics are supported for dedicated MirrorMaker clusters as of version 3.5.0. + +For new MirrorMaker clusters, set the `exactly.once.source.support` property to enabled for all targeted Kafka clusters that should be written to with exactly-once semantics. For example, to enable exactly-once for writes to cluster `us-east`, the following configuration can be used: + + + us-east.exactly.once.source.support = enabled + + +For existing MirrorMaker clusters, a two-step upgrade is necessary. Instead of immediately setting the `exactly.once.source.support` property to enabled, first set it to `preparing` on all nodes in the cluster. Once this is complete, it can be set to `enabled` on all nodes in the cluster, in a second round of restarts. + +In either case, it is also necessary to enable intra-cluster communication between the MirrorMaker nodes, as described in [KIP-710](https://cwiki.apache.org/confluence/display/KAFKA/KIP-710%3A+Full+support+for+distributed+mode+in+dedicated+MirrorMaker+2.0+clusters). To do this, the `dedicated.mode.enable.internal.rest` property must be set to `true`. In addition, many of the REST-related [configuration properties available for Kafka Connect](https://kafka.apache.org/#connectconfigs) can be specified the MirrorMaker config. For example, to enable intra-cluster communication in MirrorMaker cluster with each node listening on port 8080 of their local machine, the following should be added to the MirrorMaker config file: + + + dedicated.mode.enable.internal.rest = true + listeners = http://localhost:8080 + + +**Note that, if intra-cluster communication is enabled in production environments, it is highly recommended to secure the REST servers brought up by each MirrorMaker node. See the[configuration properties for Kafka Connect](https://kafka.apache.org/#connectconfigs) for information on how this can be accomplished. ** + +It is also recommended to filter records from aborted transactions out from replicated data when running MirrorMaker. To do this, ensure that the consumer used to read from source clusters is configured with `isolation.level` set to `read_committed`. If replicating data from cluster `us-west`, this can be done for all replication flows that read from that cluster by adding the following to the MirrorMaker config file: + + + us-west.consumer.isolation.level = read_committed + + +As a final note, under the hood, MirrorMaker uses Kafka Connect source connectors to replicate data. For more information on exactly-once support for these kinds of connectors, see the [relevant docs page](https://kafka.apache.org/#connect_exactlyoncesource). + +### Creating and Enabling Replication Flows + +To define a replication flow, you must first define the respective source and target Kafka clusters in the MirrorMaker configuration file. + + * `clusters` (required): comma-separated list of Kafka cluster "aliases" + * `{clusterAlias}.bootstrap.servers` (required): connection information for the specific cluster; comma-separated list of "bootstrap" Kafka brokers + + +Example: Define two cluster aliases `primary` and `secondary`, including their connection information. + + + clusters = primary, secondary + primary.bootstrap.servers = broker10-primary:9092,broker-11-primary:9092 + secondary.bootstrap.servers = broker5-secondary:9092,broker6-secondary:9092 + + +Secondly, you must explicitly enable individual replication flows with `{source}->{target}.enabled = true` as needed. Remember that flows are directional: if you need two-way (bidirectional) replication, you must enable flows in both directions. + + + # Enable replication from primary to secondary + primary->secondary.enabled = true + + +By default, a replication flow will replicate all but a few special topics and consumer groups from the source cluster to the target cluster, and automatically detect any newly created topics and groups. The names of replicated topics in the target cluster will be prefixed with the name of the source cluster (see section further below). For example, the topic `foo` in the source cluster `us-west` would be replicated to a topic named `us-west.foo` in the target cluster `us-east`. + +The subsequent sections explain how to customize this basic setup according to your needs. + +### Configuring Replication Flows + +The configuration of a replication flow is a combination of top-level default settings (e.g., `topics`), on top of which flow-specific settings, if any, are applied (e.g., `us-west->us-east.topics`). To change the top-level defaults, add the respective top-level setting to the MirrorMaker configuration file. To override the defaults for a specific replication flow only, use the syntax format `{source}->{target}.{config.name}`. + +The most important settings are: + + * `topics`: list of topics or a regular expression that defines which topics in the source cluster to replicate (default: `topics = .*`) + * `topics.exclude`: list of topics or a regular expression to subsequently exclude topics that were matched by the `topics` setting (default: `topics.exclude = .*[\-\.]internal, .*\.replica, __.*`) + * `groups`: list of topics or regular expression that defines which consumer groups in the source cluster to replicate (default: `groups = .*`) + * `groups.exclude`: list of topics or a regular expression to subsequently exclude consumer groups that were matched by the `groups` setting (default: `groups.exclude = console-consumer-.*, connect-.*, __.*`) + * `{source}->{target}.enable`: set to `true` to enable the replication flow (default: `false`) + + +Example: + + + # Custom top-level defaults that apply to all replication flows + topics = .* + groups = consumer-group1, consumer-group2 + + # Don't forget to enable a flow! + us-west->us-east.enabled = true + + # Custom settings for specific replication flows + us-west->us-east.topics = foo.* + us-west->us-east.groups = bar.* + us-west->us-east.emit.heartbeats = false + + +Additional configuration settings are supported which can be left with their default values in most cases. See [MirrorMaker Configs](/#mirrormakerconfigs). + +### Securing Replication Flows + +MirrorMaker supports the same security settings as Kafka Connect, so please refer to the linked section for further information. + +Example: Encrypt communication between MirrorMaker and the `us-east` cluster. + + + us-east.security.protocol=SSL + us-east.ssl.truststore.location=/path/to/truststore.jks + us-east.ssl.truststore.password=my-secret-password + us-east.ssl.keystore.location=/path/to/keystore.jks + us-east.ssl.keystore.password=my-secret-password + us-east.ssl.key.password=my-secret-password + + +### Custom Naming of Replicated Topics in Target Clusters + +Replicated topics in a target cluster—sometimes called _remote_ topics—are renamed according to a replication policy. MirrorMaker uses this policy to ensure that events (aka records, messages) from different clusters are not written to the same topic-partition. By default as per [DefaultReplicationPolicy](https://github.com/apache/kafka/blob/trunk/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/DefaultReplicationPolicy.java), the names of replicated topics in the target clusters have the format `{source}.{source_topic_name}`: + + + us-west us-east + ========= ================= + bar-topic + foo-topic --> us-west.foo-topic + + +You can customize the separator (default: `.`) with the `replication.policy.separator` setting: + + + # Defining a custom separator + us-west->us-east.replication.policy.separator = _ + + +If you need further control over how replicated topics are named, you can implement a custom `ReplicationPolicy` and override `replication.policy.class` (default is `DefaultReplicationPolicy`) in the MirrorMaker configuration. + +### Preventing Configuration Conflicts + +MirrorMaker processes share configuration via their target Kafka clusters. This behavior may cause conflicts when configurations differ among MirrorMaker processes that operate against the same target cluster. + +For example, the following two MirrorMaker processes would be racy: + + + # Configuration of process 1 + A->B.enabled = true + A->B.topics = foo + + # Configuration of process 2 + A->B.enabled = true + A->B.topics = bar + + +In this case, the two processes will share configuration via cluster `B`, which causes a conflict. Depending on which of the two processes is the elected "leader", the result will be that either the topic `foo` or the topic `bar` is replicated, but not both. + +It is therefore important to keep the MirrorMaker configuration consistent across replication flows to the same target cluster. This can be achieved, for example, through automation tooling or by using a single, shared MirrorMaker configuration file for your entire organization. + +### Best Practice: Consume from Remote, Produce to Local + +To minimize latency ("producer lag"), it is recommended to locate MirrorMaker processes as close as possible to their target clusters, i.e., the clusters that it produces data to. That's because Kafka producers typically struggle more with unreliable or high-latency network connections than Kafka consumers. + + + First DC Second DC + ========== ========================= + primary --------- MirrorMaker --> secondary + (remote) (local) + + +To run such a "consume from remote, produce to local" setup, run the MirrorMaker processes close to and preferably in the same location as the target clusters, and explicitly set these "local" clusters in the `--clusters` command line parameter (blank-separated list of cluster aliases): + + + # Run in secondary's data center, reading from the remote `primary` cluster + $ ./bin/connect-mirror-maker.sh connect-mirror-maker.properties --clusters secondary + + +The `--clusters secondary` tells the MirrorMaker process that the given cluster(s) are nearby, and prevents it from replicating data or sending configuration to clusters at other, remote locations. + +### Example: Active/Passive High Availability Deployment + +The following example shows the basic settings to replicate topics from a primary to a secondary Kafka environment, but not from the secondary back to the primary. Please be aware that most production setups will need further configuration, such as security settings. + + + # Unidirectional flow (one-way) from primary to secondary cluster + primary.bootstrap.servers = broker1-primary:9092 + secondary.bootstrap.servers = broker2-secondary:9092 + + primary->secondary.enabled = true + secondary->primary.enabled = false + + primary->secondary.topics = foo.* # only replicate some topics + + +### Example: Active/Active High Availability Deployment + +The following example shows the basic settings to replicate topics between two clusters in both ways. Please be aware that most production setups will need further configuration, such as security settings. + + + # Bidirectional flow (two-way) between us-west and us-east clusters + clusters = us-west, us-east + us-west.bootstrap.servers = broker1-west:9092,broker2-west:9092 + Us-east.bootstrap.servers = broker3-east:9092,broker4-east:9092 + + us-west->us-east.enabled = true + us-east->us-west.enabled = true + + +_Note on preventing replication "loops" (where topics will be originally replicated from A to B, then the replicated topics will be replicated yet again from B to A, and so forth)_ : As long as you define the above flows in the same MirrorMaker configuration file, you do not need to explicitly add `topics.exclude` settings to prevent replication loops between the two clusters. + +### Example: Multi-Cluster Geo-Replication + +Let's put all the information from the previous sections together in a larger example. Imagine there are three data centers (west, east, north), with two Kafka clusters in each data center (e.g., `west-1`, `west-2`). The example in this section shows how to configure MirrorMaker (1) for Active/Active replication within each data center, as well as (2) for Cross Data Center Replication (XDCR). + +First, define the source and target clusters along with their replication flows in the configuration: + + + # Basic settings + clusters: west-1, west-2, east-1, east-2, north-1, north-2 + west-1.bootstrap.servers = ... + west-2.bootstrap.servers = ... + east-1.bootstrap.servers = ... + east-2.bootstrap.servers = ... + north-1.bootstrap.servers = ... + north-2.bootstrap.servers = ... + + # Replication flows for Active/Active in West DC + west-1->west-2.enabled = true + west-2->west-1.enabled = true + + # Replication flows for Active/Active in East DC + east-1->east-2.enabled = true + east-2->east-1.enabled = true + + # Replication flows for Active/Active in North DC + north-1->north-2.enabled = true + north-2->north-1.enabled = true + + # Replication flows for XDCR via west-1, east-1, north-1 + west-1->east-1.enabled = true + west-1->north-1.enabled = true + east-1->west-1.enabled = true + east-1->north-1.enabled = true + north-1->west-1.enabled = true + north-1->east-1.enabled = true + + +Then, in each data center, launch one or more MirrorMaker as follows: + + + # In West DC: + $ ./bin/connect-mirror-maker.sh connect-mirror-maker.properties --clusters west-1 west-2 + + # In East DC: + $ ./bin/connect-mirror-maker.sh connect-mirror-maker.properties --clusters east-1 east-2 + + # In North DC: + $ ./bin/connect-mirror-maker.sh connect-mirror-maker.properties --clusters north-1 north-2 + + +With this configuration, records produced to any cluster will be replicated within the data center, as well as across to other data centers. By providing the `--clusters` parameter, we ensure that each MirrorMaker process produces data to nearby clusters only. + +_Note:_ The `--clusters` parameter is, technically, not required here. MirrorMaker will work fine without it. However, throughput may suffer from "producer lag" between data centers, and you may incur unnecessary data transfer costs. + +## Starting Geo-Replication + +You can run as few or as many MirrorMaker processes (think: nodes, servers) as needed. Because MirrorMaker is based on Kafka Connect, MirrorMaker processes that are configured to replicate the same Kafka clusters run in a distributed setup: They will find each other, share configuration (see section below), load balance their work, and so on. If, for example, you want to increase the throughput of replication flows, one option is to run additional MirrorMaker processes in parallel. + +To start a MirrorMaker process, run the command: + + + $ ./bin/connect-mirror-maker.sh connect-mirror-maker.properties + + +After startup, it may take a few minutes until a MirrorMaker process first begins to replicate data. + +Optionally, as described previously, you can set the parameter `--clusters` to ensure that the MirrorMaker process produces data to nearby clusters only. + + + # Note: The cluster alias us-west must be defined in the configuration file + $ ./bin/connect-mirror-maker.sh connect-mirror-maker.properties \ + --clusters us-west + + +_Note when testing replication of consumer groups:_ By default, MirrorMaker does not replicate consumer groups created by the `kafka-console-consumer.sh` tool, which you might use to test your MirrorMaker setup on the command line. If you do want to replicate these consumer groups as well, set the `groups.exclude` configuration accordingly (default: `groups.exclude = console-consumer-.*, connect-.*, __.*`). Remember to update the configuration again once you completed your testing. + +## Stopping Geo-Replication + +You can stop a running MirrorMaker process by sending a SIGTERM signal with the command: + + + $ kill + + +## Applying Configuration Changes + +To make configuration changes take effect, the MirrorMaker process(es) must be restarted. + +## Monitoring Geo-Replication + +It is recommended to monitor MirrorMaker processes to ensure all defined replication flows are up and running correctly. MirrorMaker is built on the Connect framework and inherits all of Connect's metrics, such `source-record-poll-rate`. In addition, MirrorMaker produces its own metrics under the `kafka.connect.mirror` metric group. Metrics are tagged with the following properties: + + * `source`: alias of source cluster (e.g., `primary`) + * `target`: alias of target cluster (e.g., `secondary`) + * `topic`: replicated topic on target cluster + * `partition`: partition being replicated + + + +Metrics are tracked for each replicated topic. The source cluster can be inferred from the topic name. For example, replicating `topic1` from `primary->secondary` will yield metrics like: + + * `target=secondary` + * `topic=primary.topic1` + * `partition=1` + + +The following metrics are emitted: + + + # MBean: kafka.connect.mirror:type=MirrorSourceConnector,target=([-.w]+),topic=([-.w]+),partition=([0-9]+) + + record-count # number of records replicated source -> target + record-age-ms # age of records when they are replicated + record-age-ms-min + record-age-ms-max + record-age-ms-avg + replication-latency-ms # time it takes records to propagate source->target + replication-latency-ms-min + replication-latency-ms-max + replication-latency-ms-avg + byte-rate # average number of bytes/sec in replicated records + + # MBean: kafka.connect.mirror:type=MirrorCheckpointConnector,source=([-.w]+),target=([-.w]+) + + checkpoint-latency-ms # time it takes to replicate consumer offsets + checkpoint-latency-ms-min + checkpoint-latency-ms-max + checkpoint-latency-ms-avg + + +These metrics do not differentiate between created-at and log-append timestamps. diff --git a/docs/operations/hardware-and-os.md b/docs/operations/hardware-and-os.md new file mode 100644 index 0000000000000..23fde803fd3f8 --- /dev/null +++ b/docs/operations/hardware-and-os.md @@ -0,0 +1,131 @@ +--- +title: Hardware and OS +description: Hardware and OS +weight: 7 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +We are using dual quad-core Intel Xeon machines with 24GB of memory. + +You need sufficient memory to buffer active readers and writers. You can do a back-of-the-envelope estimate of memory needs by assuming you want to be able to buffer for 30 seconds and compute your memory need as write_throughput*30. + +The disk throughput is important. We have 8x7200 rpm SATA drives. In general disk throughput is the performance bottleneck, and more disks is better. Depending on how you configure flush behavior you may or may not benefit from more expensive disks (if you force flush often then higher RPM SAS drives may be better). + +## OS + +Kafka should run well on any unix system and has been tested on Linux and Solaris. + +We have seen a few issues running on Windows and Windows is not currently a well supported platform though we would be happy to change that. + +It is unlikely to require much OS-level tuning, but there are three potentially important OS-level configurations: + + * File descriptor limits: Kafka uses file descriptors for log segments and open connections. If a broker hosts many partitions, consider that the broker needs at least (number_of_partitions)*(partition_size/segment_size) to track all log segments in addition to the number of connections the broker makes. We recommend at least 100000 allowed file descriptors for the broker processes as a starting point. Note: The mmap() function adds an extra reference to the file associated with the file descriptor fildes which is not removed by a subsequent close() on that file descriptor. This reference is removed when there are no more mappings to the file. + * Max socket buffer size: can be increased to enable high-performance data transfer between data centers as [described here](https://www.psc.edu/index.php/networking/641-tcp-tune). + * Maximum number of memory map areas a process may have (aka vm.max_map_count). [See the Linux kernel documentation](https://kernel.org/doc/Documentation/sysctl/vm.txt). You should keep an eye at this OS-level property when considering the maximum number of partitions a broker may have. By default, on a number of Linux systems, the value of vm.max_map_count is somewhere around 65535. Each log segment, allocated per partition, requires a pair of index/timeindex files, and each of these files consumes 1 map area. In other words, each log segment uses 2 map areas. Thus, each partition requires minimum 2 map areas, as long as it hosts a single log segment. That is to say, creating 50000 partitions on a broker will result allocation of 100000 map areas and likely cause broker crash with OutOfMemoryError (Map failed) on a system with default vm.max_map_count. Keep in mind that the number of log segments per partition varies depending on the segment size, load intensity, retention policy and, generally, tends to be more than one. + + +## Disks and Filesystem + +We recommend using multiple drives to get good throughput and not sharing the same drives used for Kafka data with application logs or other OS filesystem activity to ensure good latency. You can either RAID these drives together into a single volume or format and mount each drive as its own directory. Since Kafka has replication the redundancy provided by RAID can also be provided at the application level. This choice has several tradeoffs. + +If you configure multiple data directories partitions will be assigned round-robin to data directories. Each partition will be entirely in one of the data directories. If data is not well balanced among partitions this can lead to load imbalance between disks. + +RAID can potentially do better at balancing load between disks (although it doesn't always seem to) because it balances load at a lower level. The primary downside of RAID is that it is usually a big performance hit for write throughput and reduces the available disk space. + +Another potential benefit of RAID is the ability to tolerate disk failures. However our experience has been that rebuilding the RAID array is so I/O intensive that it effectively disables the server, so this does not provide much real availability improvement. + +## Application vs. OS Flush Management + +Kafka always immediately writes all data to the filesystem and supports the ability to configure the flush policy that controls when data is forced out of the OS cache and onto disk using the flush. This flush policy can be controlled to force data to disk after a period of time or after a certain number of messages has been written. There are several choices in this configuration. + +Kafka must eventually call fsync to know that data was flushed. When recovering from a crash for any log segment not known to be fsync'd Kafka will check the integrity of each message by checking its CRC and also rebuild the accompanying offset index file as part of the recovery process executed on startup. + +Note that durability in Kafka does not require syncing data to disk, as a failed node will always recover from its replicas. + +We recommend using the default flush settings which disable application fsync entirely. This means relying on the background flush done by the OS and Kafka's own background flush. This provides the best of all worlds for most uses: no knobs to tune, great throughput and latency, and full recovery guarantees. We generally feel that the guarantees provided by replication are stronger than sync to local disk, however the paranoid still may prefer having both and application level fsync policies are still supported. + +The drawback of using application level flush settings is that it is less efficient in its disk usage pattern (it gives the OS less leeway to re-order writes) and it can introduce latency as fsync in most Linux filesystems blocks writes to the file whereas the background flushing does much more granular page-level locking. + +In general you don't need to do any low-level tuning of the filesystem, but in the next few sections we will go over some of this in case it is useful. + +## Understanding Linux OS Flush Behavior + +In Linux, data written to the filesystem is maintained in [pagecache](https://en.wikipedia.org/wiki/Page_cache) until it must be written out to disk (due to an application-level fsync or the OS's own flush policy). The flushing of data is done by a set of background threads called pdflush (or in post 2.6.32 kernels "flusher threads"). + +Pdflush has a configurable policy that controls how much dirty data can be maintained in cache and for how long before it must be written back to disk. This policy is described [here](https://web.archive.org/web/20160518040713/http://www.westnet.com/~gsmith/content/linux-pdflush.htm). When Pdflush cannot keep up with the rate of data being written it will eventually cause the writing process to block incurring latency in the writes to slow down the accumulation of data. + +You can see the current state of OS memory usage by doing + + + > cat /proc/meminfo + +The meaning of these values are described in the link above. + +Using pagecache has several advantages over an in-process cache for storing data that will be written out to disk: + + * The I/O scheduler will batch together consecutive small writes into bigger physical writes which improves throughput. + * The I/O scheduler will attempt to re-sequence writes to minimize movement of the disk head which improves throughput. + * It automatically uses all the free memory on the machine + + +## Filesystem Selection + +Kafka uses regular files on disk, and as such it has no hard dependency on a specific filesystem. The two filesystems which have the most usage, however, are EXT4 and XFS. Historically, EXT4 has had more usage, but recent improvements to the XFS filesystem have shown it to have better performance characteristics for Kafka's workload with no compromise in stability. + +Comparison testing was performed on a cluster with significant message loads, using a variety of filesystem creation and mount options. The primary metric in Kafka that was monitored was the "Request Local Time", indicating the amount of time append operations were taking. XFS resulted in much better local times (160ms vs. 250ms+ for the best EXT4 configuration), as well as lower average wait times. The XFS performance also showed less variability in disk performance. + +### General Filesystem Notes + +For any filesystem used for data directories, on Linux systems, the following options are recommended to be used at mount time: + + * noatime: This option disables updating of a file's atime (last access time) attribute when the file is read. This can eliminate a significant number of filesystem writes, especially in the case of bootstrapping consumers. Kafka does not rely on the atime attributes at all, so it is safe to disable this. + + + +### XFS Notes + +The XFS filesystem has a significant amount of auto-tuning in place, so it does not require any change in the default settings, either at filesystem creation time or at mount. The only tuning parameters worth considering are: + + * largeio: This affects the preferred I/O size reported by the stat call. While this can allow for higher performance on larger disk writes, in practice it had minimal or no effect on performance. + * nobarrier: For underlying devices that have battery-backed cache, this option can provide a little more performance by disabling periodic write flushes. However, if the underlying device is well-behaved, it will report to the filesystem that it does not require flushes, and this option will have no effect. + + + +### EXT4 Notes + +EXT4 is a serviceable choice of filesystem for the Kafka data directories, however getting the most performance out of it will require adjusting several mount options. In addition, these options are generally unsafe in a failure scenario, and will result in much more data loss and corruption. For a single broker failure, this is not much of a concern as the disk can be wiped and the replicas rebuilt from the cluster. In a multiple-failure scenario, such as a power outage, this can mean underlying filesystem (and therefore data) corruption that is not easily recoverable. The following options can be adjusted: + + * data=writeback: Ext4 defaults to data=ordered which puts a strong order on some writes. Kafka does not require this ordering as it does very paranoid data recovery on all unflushed log. This setting removes the ordering constraint and seems to significantly reduce latency. + * Disabling journaling: Journaling is a tradeoff: it makes reboots faster after server crashes but it introduces a great deal of additional locking which adds variance to write performance. Those who don't care about reboot time and want to reduce a major source of write latency spikes can turn off journaling entirely. + * commit=num_secs: This tunes the frequency with which ext4 commits to its metadata journal. Setting this to a lower value reduces the loss of unflushed data during a crash. Setting this to a higher value will improve throughput. + * nobh: This setting controls additional ordering guarantees when using data=writeback mode. This should be safe with Kafka as we do not depend on write ordering and improves throughput and latency. + * delalloc: Delayed allocation means that the filesystem avoid allocating any blocks until the physical write occurs. This allows ext4 to allocate a large extent instead of smaller pages and helps ensure the data is written sequentially. This feature is great for throughput. It does seem to involve some locking in the filesystem which adds a bit of latency variance. + + +## Replace KRaft Controller Disk + +When Kafka is configured to use KRaft, the controllers store the cluster metadata in the directory specified in `metadata.log.dir` \-- or the first log directory, if `metadata.log.dir` is not configured. See the documentation for `metadata.log.dir` for details. + +If the data in the cluster metadata directory is lost either because of hardware failure or the hardware needs to be replaced, care should be taken when provisioning the new controller node. The new controller node should not be formatted and started until the majority of the controllers have all of the committed data. To determine if the majority of the controllers have the committed data, run the `kafka-metadata-quorum.sh` tool to describe the replication status: + + + > bin/kafka-metadata-quorum.sh --bootstrap-server broker_host:port describe --replication + NodeId LogEndOffset Lag LastFetchTimestamp LastCaughtUpTimestamp Status + 1 25806 0 1662500992757 1662500992757 Leader + ... ... ... ... ... ... + + +Check and wait until the `Lag` is small for a majority of the controllers. If the leader's end offset is not increasing, you can wait until the lag is 0 for a majority; otherwise, you can pick the latest leader end offset and wait until all replicas have reached it. Check and wait until the `LastFetchTimestamp` and `LastCaughtUpTimestamp` are close to each other for the majority of the controllers. At this point it is safer to format the controller's metadata log directory. This can be done by running the `kafka-storage.sh` command. + + + > bin/kafka-storage.sh format --cluster-id uuid --config server_properties + +It is possible for the `bin/kafka-storage.sh format` command above to fail with a message like `Log directory ... is already formatted`. This can happen when combined mode is used and only the metadata log directory was lost but not the others. In that case and only in that case, can you run the `kafka-storage.sh format` command with the `--ignore-formatted` option. + +Start the KRaft controller after formatting the log directories. + + + > /bin/kafka-server-start.sh server_properties diff --git a/docs/operations/java-version.md b/docs/operations/java-version.md new file mode 100644 index 0000000000000..f4513a6c94ce3 --- /dev/null +++ b/docs/operations/java-version.md @@ -0,0 +1,33 @@ +--- +title: Java Version +description: Java Version +weight: 6 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +Java 8, Java 11, and Java 17 are supported. + +Note that Java 8 support project-wide has been deprecated since Apache Kafka 3.0 and Java 11 support for the broker and tools has been deprecated since Apache Kafka 3.7. Both will be removed in Apache Kafka 4.0. + +Java 11 and later versions perform significantly better if TLS is enabled, so they are highly recommended (they also include a number of other performance improvements: G1GC, CRC32C, Compact Strings, Thread-Local Handshakes and more). + +From a security perspective, we recommend the latest released patch version as older freely available versions have disclosed security vulnerabilities. + +Typical arguments for running Kafka with OpenJDK-based Java implementations (including Oracle JDK) are: + + + -Xmx6g -Xms6g -XX:MetaspaceSize=96m -XX:+UseG1GC + -XX:MaxGCPauseMillis=20 -XX:InitiatingHeapOccupancyPercent=35 -XX:G1HeapRegionSize=16M + -XX:MinMetaspaceFreeRatio=50 -XX:MaxMetaspaceFreeRatio=80 -XX:+ExplicitGCInvokesConcurrent + +For reference, here are the stats for one of LinkedIn's busiest clusters (at peak) that uses said Java arguments: + + * 60 brokers + * 50k partitions (replication factor 2) + * 800k messages/sec in + * 300 MB/sec inbound, 1 GB/sec+ outbound + +All of the brokers in that cluster have a 90% GC pause time of about 21ms with less than 1 young GC per second. diff --git a/docs/operations/kafka-configuration.md b/docs/operations/kafka-configuration.md new file mode 100644 index 0000000000000..7fa75399d3a22 --- /dev/null +++ b/docs/operations/kafka-configuration.md @@ -0,0 +1,43 @@ +--- +title: Kafka Configuration +description: Kafka Configuration +weight: 5 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +## Important Client Configurations + +The most important producer configurations are: + + * acks + * compression + * batch size + +The most important consumer configuration is the fetch size. + +All configurations are documented in the configuration section. + +## A Production Server Config + +Here is an example production server configuration: + + + # ZooKeeper + zookeeper.connect=[list of ZooKeeper servers] + + # Log configuration + num.partitions=8 + default.replication.factor=3 + log.dir=[List of directories. Kafka should have its own dedicated disk(s) or SSD(s).] + + # Other configurations + broker.id=[An integer. Start with 0 and increment by 1 for each new broker.] + listeners=[list of listeners] + auto.create.topics.enable=false + min.insync.replicas=2 + queued.max.requests=[number of concurrent requests] + +Our client configuration varies a fair amount between different use cases. diff --git a/docs/operations/kraft.md b/docs/operations/kraft.md new file mode 100644 index 0000000000000..f84c15c8fbad6 --- /dev/null +++ b/docs/operations/kraft.md @@ -0,0 +1,405 @@ +--- +title: KRaft +description: KRaft +weight: 10 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +## Configuration + +### Process Roles + +In KRaft mode each Kafka server can be configured as a controller, a broker, or both using the `process.roles` property. This property can have the following values: + + * If `process.roles` is set to `broker`, the server acts as a broker. + * If `process.roles` is set to `controller`, the server acts as a controller. + * If `process.roles` is set to `broker,controller`, the server acts as both a broker and a controller. + * If `process.roles` is not set at all, it is assumed to be in ZooKeeper mode. + + + +Kafka servers that act as both brokers and controllers are referred to as "combined" servers. Combined servers are simpler to operate for small use cases like a development environment. The key disadvantage is that the controller will be less isolated from the rest of the system. For example, it is not possible to roll or scale the controllers separately from the brokers in combined mode. Combined mode is not recommended in critical deployment environments. + +### Controllers + +In KRaft mode, specific Kafka servers are selected to be controllers (unlike the ZooKeeper-based mode, where any server can become the Controller). The servers selected to be controllers will participate in the metadata quorum. Each controller is either an active or a hot standby for the current active controller. + +A Kafka admin will typically select 3 or 5 servers for this role, depending on factors like cost and the number of concurrent failures your system should withstand without availability impact. A majority of the controllers must be alive in order to maintain availability. With 3 controllers, the cluster can tolerate 1 controller failure; with 5 controllers, the cluster can tolerate 2 controller failures. + +All of the servers in a Kafka cluster discover the quorum voters using the `controller.quorum.voters` property. This identifies the quorum controller servers that should be used. All the controllers must be enumerated. Each controller is identified with their `id`, `host` and `port` information. For example: + + + controller.quorum.voters=id1@host1:port1,id2@host2:port2,id3@host3:port3 + +If a Kafka cluster has 3 controllers named controller1, controller2 and controller3, then controller1 may have the following configuration: + + + process.roles=controller + node.id=1 + listeners=CONTROLLER://controller1.example.com:9093 + controller.quorum.voters=1@controller1.example.com:9093,2@controller2.example.com:9093,3@controller3.example.com:9093 + +Every broker and controller must set the `controller.quorum.voters` property. The node ID supplied in the `controller.quorum.voters` property must match the corresponding id on the controller servers. For example, on controller1, node.id must be set to 1, and so forth. Each node ID must be unique across all the servers in a particular cluster. No two servers can have the same node ID regardless of their `process.roles` values. + +## Storage Tool + +The `kafka-storage.sh random-uuid` command can be used to generate a cluster ID for your new cluster. This cluster ID must be used when formatting each server in the cluster with the `kafka-storage.sh format` command. + +This is different from how Kafka has operated in the past. Previously, Kafka would format blank storage directories automatically, and also generate a new cluster ID automatically. One reason for the change is that auto-formatting can sometimes obscure an error condition. This is particularly important for the metadata log maintained by the controller and broker servers. If a majority of the controllers were able to start with an empty log directory, a leader might be able to be elected with missing committed data. + +## Debugging + +### Metadata Quorum Tool + +The `kafka-metadata-quorum` tool can be used to describe the runtime state of the cluster metadata partition. For example, the following command displays a summary of the metadata quorum: + + + > bin/kafka-metadata-quorum.sh --bootstrap-server broker_host:port describe --status + ClusterId: fMCL8kv1SWm87L_Md-I2hg + LeaderId: 3002 + LeaderEpoch: 2 + HighWatermark: 10 + MaxFollowerLag: 0 + MaxFollowerLagTimeMs: -1 + CurrentVoters: [3000,3001,3002] + CurrentObservers: [0,1,2] + +### Dump Log Tool + +The `kafka-dump-log` tool can be used to debug the log segments and snapshots for the cluster metadata directory. The tool will scan the provided files and decode the metadata records. For example, this command decodes and prints the records in the first log segment: + + + > bin/kafka-dump-log.sh --cluster-metadata-decoder --files metadata_log_dir/__cluster_metadata-0/00000000000000000000.log + +This command decodes and prints the records in the a cluster metadata snapshot: + + + > bin/kafka-dump-log.sh --cluster-metadata-decoder --files metadata_log_dir/__cluster_metadata-0/00000000000000000100-0000000001.checkpoint + +### Metadata Shell + +The `kafka-metadata-shell` tool can be used to interactively inspect the state of the cluster metadata partition: + + + > bin/kafka-metadata-shell.sh --snapshot metadata_log_dir/__cluster_metadata-0/00000000000000000000.log + >> ls / + brokers local metadataQuorum topicIds topics + >> ls /topics + foo + >> cat /topics/foo/0/data + { + "partitionId" : 0, + "topicId" : "5zoAlv-xEh9xRANKXt1Lbg", + "replicas" : [ 1 ], + "isr" : [ 1 ], + "removingReplicas" : null, + "addingReplicas" : null, + "leader" : 1, + "leaderEpoch" : 0, + "partitionEpoch" : 0 + } + >> exit + + +## Deploying Considerations + + * Kafka server's `process.role` should be set to either `broker` or `controller` but not both. Combined mode can be used in development environments, but it should be avoided in critical deployment environments. + * For redundancy, a Kafka cluster should use 3 controllers. More than 3 controllers is not recommended in critical environments. In the rare case of a partial network failure it is possible for the cluster metadata quorum to become unavailable. This limitation will be addressed in a future release of Kafka. + * The Kafka controllers store all the metadata for the cluster in memory and on disk. We believe that for a typical Kafka cluster 5GB of main memory and 5GB of disk space on the metadata log director is sufficient. + + + +## Missing Features + +The following features are not fully implemented in KRaft mode: + + * Supporting JBOD configurations with multiple storage directories. Note that an Early Access release is supported in 3.7 as per [KIP-858](https://cwiki.apache.org/confluence/display/KAFKA/KIP-858%3A+Handle+JBOD+broker+disk+failure+in+KRaft). Note that it is not yet recommended for use in production environments. Please refer to the [release notes](https://cwiki.apache.org/confluence/display/KAFKA/Kafka+JBOD+in+KRaft+Early+Access+Release+Notes) to help us test it and provide feedback at [KAFKA-16061](https://issues.apache.org/jira/browse/KAFKA-16061). + * Modifying certain dynamic configurations on the standalone KRaft controller + + + +## ZooKeeper to KRaft Migration + +**ZooKeeper to KRaft migration is considered an Early Access feature and is not recommended for production clusters.** Please report issues with ZooKeeper to KRaft migration using the [project JIRA](https://issues.apache.org/jira/projects/KAFKA) and the "kraft" component. + +### Terminology + + * Brokers that are in **ZK mode** store their metadata in Apache ZooKepeer. This is the old mode of handling metadata. + * Brokers that are in **KRaft mode** store their metadata in a KRaft quorum. This is the new and improved mode of handling metadata. + * **Migration** is the process of moving cluster metadata from ZooKeeper into a KRaft quorum. + + + +### Migration Phases + +In general, the migration process passes through several phases. + + * In the **initial phase** , all the brokers are in ZK mode, and there is a ZK-based controller. + * During the **initial metadata load** , a KRaft quorum loads the metadata from ZooKeeper, + * In **hybrid phase** , some brokers are in ZK mode, but there is a KRaft controller. + * In **dual-write phase** , all brokers are KRaft, but the KRaft controller is continuing to write to ZK. + * When the migration has been **finalized** , we no longer write metadata to ZooKeeper. + + + +### Limitations + + * While a cluster is being migrated from ZK mode to KRaft mode, we do not support changing the _metadata version_ (also known as the _inter.broker.protocol_ version.) Please do not attempt to do this during a migration, or you may break the cluster. + * After the migration has been finalized, it is not possible to revert back to ZooKeeper mode. + * As noted above, some features are not fully implemented in KRaft mode. If you are using one of those features, you will not be able to migrate to KRaft yet. + + + +### Preparing for migration + +Before beginning the migration, the Kafka brokers must be upgraded to software version 3.7.2 and have the "inter.broker.protocol.version" configuration set to "3.7". + +It is recommended to enable TRACE level logging for the migration components while the migration is active. This can be done by adding the following log4j configuration to each KRaft controller's "log4j.properties" file. + + + log4j.logger.org.apache.kafka.metadata.migration=TRACE + +It is generally useful to enable DEBUG logging on the KRaft controllers and the ZK brokers during the migration. + +### Provisioning the KRaft controller quorum + +Two things are needed before the migration can begin. First, the brokers must be configured to support the migration and second, a KRaft controller quorum must be deployed. The KRaft controllers should be provisioned with the same cluster ID as the existing Kafka cluster. This can be found by examining one of the "meta.properties" files in the data directories of the brokers, or by running the following command. + + + ./bin/zookeeper-shell.sh localhost:2181 get /cluster/id + +The KRaft controller quorum should also be provisioned with the latest `metadata.version`. This is done automatically when you format the node with the `kafka-storage.sh` tool. For further instructions on KRaft deployment, please refer to the above documentation. + +In addition to the standard KRaft configuration, the KRaft controllers will need to enable support for the migration as well as provide ZooKeeper connection configuration. + +Here is a sample config for a KRaft controller that is ready for migration: + + + # Sample KRaft cluster controller.properties listening on 9093 + process.roles=controller + node.id=3000 + controller.quorum.voters=3000@localhost:9093 + controller.listener.names=CONTROLLER + listeners=CONTROLLER://:9093 + + # Enable the migration + zookeeper.metadata.migration.enable=true + + # ZooKeeper client configuration + zookeeper.connect=localhost:2181 + + # The inter broker listener in brokers to allow KRaft controller send RPCs to brokers + inter.broker.listener.name=PLAINTEXT + + # Other configs ... + +Note: The migration can stall if the ZooKeeper Security Migration Tool was previously executed (see [KAFKA-19480](https://issues.apache.org/jira/browse/KAFKA-19480) for more details). As a workaround, the malformed "/migration" node can be removed from ZooKeeper by running `delete /migration` with the `zookeeper-shell.sh` CLI tool. + +_Note: The KRaft cluster`node.id` values must be different from any existing ZK broker `broker.id`. In KRaft-mode, the brokers and controllers share the same Node ID namespace._ + +### Enter Migration Mode on the Brokers + +Once the KRaft controller quorum has been started, the brokers will need to be reconfigured and restarted. Brokers may be restarted in a rolling fashion to avoid impacting cluster availability. Each broker requires the following configuration to communicate with the KRaft controllers and to enable the migration. + + * controller.quorum.voters + * controller.listener.names + * The controller.listener.name should also be added to listener.security.property.map + * zookeeper.metadata.migration.enable + + + +Here is a sample config for a broker that is ready for migration: + + + # Sample ZK broker server.properties listening on 9092 + broker.id=0 + listeners=PLAINTEXT://:9092 + advertised.listeners=PLAINTEXT://localhost:9092 + listener.security.protocol.map=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT + + # Set the IBP + inter.broker.protocol.version=3.7 + + # Enable the migration + zookeeper.metadata.migration.enable=true + + # ZooKeeper client configuration + zookeeper.connect=localhost:2181 + + # KRaft controller quorum configuration + controller.quorum.voters=3000@localhost:9093 + controller.listener.names=CONTROLLER + +_Note: Once the final ZK broker has been restarted with the necessary configuration, the migration will automatically begin._ When the migration is complete, an INFO level log can be observed on the active controller: + + + Completed migration of metadata from Zookeeper to KRaft + +### Migrating brokers to KRaft + +Once the KRaft controller completes the metadata migration, the brokers will still be running in ZooKeeper mode. While the KRaft controller is in migration mode, it will continue sending controller RPCs to the ZooKeeper mode brokers. This includes RPCs like UpdateMetadata and LeaderAndIsr. + +To migrate the brokers to KRaft, they simply need to be reconfigured as KRaft brokers and restarted. Using the above broker configuration as an example, we would replace the `broker.id` with `node.id` and add `process.roles=broker`. It is important that the broker maintain the same Broker/Node ID when it is restarted. The zookeeper configurations should be removed at this point. Finally, if you have set `control.plane.listener.name`. please remove it before restarting in KRaft mode. + +If your broker has authorization configured via the `authorizer.class.name` property using `kafka.security.authorizer.AclAuthorizer`, this is also the time to change it to use `org.apache.kafka.metadata.authorizer.StandardAuthorizer` instead. + + + # Sample KRaft broker server.properties listening on 9092 + process.roles=broker + node.id=0 + listeners=PLAINTEXT://:9092 + advertised.listeners=PLAINTEXT://localhost:9092 + listener.security.protocol.map=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT + + # Don't set the IBP, KRaft uses "metadata.version" feature flag + # inter.broker.protocol.version=3.7 + + # Remove the migration enabled flag + # zookeeper.metadata.migration.enable=true + + # Remove ZooKeeper client configuration + # zookeeper.connect=localhost:2181 + + # Keep the KRaft controller quorum configuration + controller.quorum.voters=3000@localhost:9093 + controller.listener.names=CONTROLLER + +Each broker is restarted with a KRaft configuration until the entire cluster is running in KRaft mode. + +### Finalizing the migration + +Once all brokers have been restarted in KRaft mode, the last step to finalize the migration is to take the KRaft controllers out of migration mode. This is done by removing the "zookeeper.metadata.migration.enable" property from each of their configs and restarting them one at a time. + +Once the migration has been finalized, you can safely deprovision your ZooKeeper cluster, assuming you are not using it for anything else. After this point, it is no longer possible to revert to ZooKeeper mode. + + + # Sample KRaft cluster controller.properties listening on 9093 + process.roles=controller + node.id=3000 + controller.quorum.voters=3000@localhost:9093 + controller.listener.names=CONTROLLER + listeners=CONTROLLER://:9093 + + # Disable the migration + # zookeeper.metadata.migration.enable=true + + # Remove ZooKeeper client configuration + # zookeeper.connect=localhost:2181 + + # Other configs ... + +### Reverting to ZooKeeper mode During the Migration + +While the cluster is still in migration mode, it is possible to revert to ZooKeeper mode. The process to follow depends on how far the migration has progressed. In order to find out how to revert, select the **final** migration step that you have **completed** in this table. + +Note that the directions given here assume that each step was fully completed, and they were done in order. So, for example, we assume that if "Enter Migration Mode on the Brokers" was completed, "Provisioning the KRaft controller quorum" was also fully completed previously. + +If you did not fully complete any step, back out whatever you have done and then follow revert directions for the last fully completed step. + + + + + + + + + + + + + + + + + + + + + + + + + +
+ +Final Migration Section Completed + + +Directions for Reverting + + +Notes +
+ +Preparing for migration + + +The preparation section does not involve leaving ZooKeeper mode. So there is nothing to do in the case of a revert. + + + +
+ +Provisioning the KRaft controller quorum + + + + + * Deprovision the KRaft controller quorum. + * Then you are done. + + + + + +
+ +Enter Migration Mode on the brokers + + + + + * Deprovision the KRaft controller quorum. + * Using `zookeeper-shell.sh`, run `rmr /controller` so that one of the brokers can become the new old-style controller. + * On each broker, remove the `zookeeper.metadata.migration.enable`, `controller.listener.names`, and `controller.quorum.voters` configurations, and replace `node.id` with `broker.id`. Then perform a rolling restart of all brokers. + * Then you are done. + + + + +It is important to perform the `zookeeper-shell.sh` step quickly, to minimize the amount of time that the cluster lacks a controller. +
+ +Migrating brokers to KRaft + + + + + * On each broker, remove the `process.roles` configuration, and restore the `zookeeper.connect` configuration to its previous value. If your cluster requires other ZooKeeper configurations for brokers, such as `zookeeper.ssl.protocol`, re-add those configurations as well. Then perform a rolling restart of all brokers. + * Deprovision the KRaft controller quorum. + * Using `zookeeper-shell.sh`, run `rmr /controller` so that one of the brokers can become the new old-style controller. + * On each broker, remove the `zookeeper.metadata.migration.enable`, `controller.listener.names`, and `controller.quorum.voters` configurations. Replace `node.id` with `broker.id`. Then perform a second rolling restart of all brokers. + * Then you are done. + + + + + + + * It is important to perform the `zookeeper-shell.sh` step **quickly** , to minimize the amount of time that the cluster lacks a controller. + * Make sure that on the first cluster roll, `zookeeper.metadata.migration.enable` remains set to `true`. **Do not set it to false until the second cluster roll.** + + +
+ +Finalizing the migration + + +If you have finalized the ZK migration, then you cannot revert. + + +Some users prefer to wait for a week or two before finalizing the migration. While this requires you to keep the ZooKeeper cluster running for a while longer, it may be helpful in validating KRaft mode in your cluster. +
diff --git a/docs/operations/monitoring.md b/docs/operations/monitoring.md new file mode 100644 index 0000000000000..3734112cad566 --- /dev/null +++ b/docs/operations/monitoring.md @@ -0,0 +1,5371 @@ +--- +title: Monitoring +description: Monitoring +weight: 8 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +Kafka uses Yammer Metrics for metrics reporting in the server. The Java clients use Kafka Metrics, a built-in metrics registry that minimizes transitive dependencies pulled into client applications. Both expose metrics via JMX and can be configured to report stats using pluggable stats reporters to hook up to your monitoring system. + +All Kafka rate metrics have a corresponding cumulative count metric with suffix `-total`. For example, `records-consumed-rate` has a corresponding metric named `records-consumed-total`. + +The easiest way to see the available metrics is to fire up jconsole and point it at a running kafka client or server; this will allow browsing all metrics with JMX. + +## Security Considerations for Remote Monitoring using JMX + +Apache Kafka disables remote JMX by default. You can enable remote monitoring using JMX by setting the environment variable `JMX_PORT` for processes started using the CLI or standard Java system properties to enable remote JMX programmatically. You must enable security when enabling remote JMX in production scenarios to ensure that unauthorized users cannot monitor or control your broker or application as well as the platform on which these are running. Note that authentication is disabled for JMX by default in Kafka and security configs must be overridden for production deployments by setting the environment variable `KAFKA_JMX_OPTS` for processes started using the CLI or by setting appropriate Java system properties. See [Monitoring and Management Using JMX Technology](https://docs.oracle.com/javase/8/docs/technotes/guides/management/agent.html) for details on securing JMX. + +We do graphing and alerting on the following metrics: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+ +Description + + +Mbean name + + +Normal value +
+ +Message in rate + + +kafka.server:type=BrokerTopicMetrics,name=MessagesInPerSec,topic=([-.\w]+) + + +Incoming message rate per topic. Omitting 'topic=(...)' will yield the all-topic rate. +
+ +Byte in rate from clients + + +kafka.server:type=BrokerTopicMetrics,name=BytesInPerSec,topic=([-.\w]+) + + +Byte in (from the clients) rate per topic. Omitting 'topic=(...)' will yield the all-topic rate. +
+ +Byte in rate from other brokers + + +kafka.server:type=BrokerTopicMetrics,name=ReplicationBytesInPerSec + + +Byte in (from the other brokers) rate across all topics. +
+ +Controller Request rate from Broker + + +kafka.controller:type=ControllerChannelManager,name=RequestRateAndQueueTimeMs,brokerId=([0-9]+) + + +The rate (requests per second) at which the ControllerChannelManager takes requests from the queue of the given broker. And the time it takes for a request to stay in this queue before it is taken from the queue. +
+ +Controller Event queue size + + +kafka.controller:type=ControllerEventManager,name=EventQueueSize + + +Size of the ControllerEventManager's queue. +
+ +Controller Event queue time + + +kafka.controller:type=ControllerEventManager,name=EventQueueTimeMs + + +Time that takes for any event (except the Idle event) to wait in the ControllerEventManager's queue before being processed +
+ +Request rate + + +kafka.network:type=RequestMetrics,name=RequestsPerSec,request={Produce|FetchConsumer|FetchFollower},version=([0-9]+) + + + +
+ +Error rate + + +kafka.network:type=RequestMetrics,name=ErrorsPerSec,request=([-.\w]+),error=([-.\w]+) + + +Number of errors in responses counted per-request-type, per-error-code. If a response contains multiple errors, all are counted. error=NONE indicates successful responses. +
+ +Produce request rate + + +kafka.server:type=BrokerTopicMetrics,name=TotalProduceRequestsPerSec,topic=([-.\w]+) + + +Produce request rate per topic. Omitting 'topic=(...)' will yield the all-topic rate. +
+ +Fetch request rate + + +kafka.server:type=BrokerTopicMetrics,name=TotalFetchRequestsPerSec,topic=([-.\w]+) + + +Fetch request (from clients or followers) rate per topic. Omitting 'topic=(...)' will yield the all-topic rate. +
+ +Failed produce request rate + + +kafka.server:type=BrokerTopicMetrics,name=FailedProduceRequestsPerSec,topic=([-.\w]+) + + +Failed Produce request rate per topic. Omitting 'topic=(...)' will yield the all-topic rate. +
+ +Failed fetch request rate + + +kafka.server:type=BrokerTopicMetrics,name=FailedFetchRequestsPerSec,topic=([-.\w]+) + + +Failed Fetch request (from clients or followers) rate per topic. Omitting 'topic=(...)' will yield the all-topic rate. +
+ +Request size in bytes + + +kafka.network:type=RequestMetrics,name=RequestBytes,request=([-.\w]+) + + +Size of requests for each request type. +
+ +Temporary memory size in bytes + + +kafka.network:type=RequestMetrics,name=TemporaryMemoryBytes,request={Produce|Fetch} + + +Temporary memory used for message format conversions and decompression. +
+ +Message conversion time + + +kafka.network:type=RequestMetrics,name=MessageConversionsTimeMs,request={Produce|Fetch} + + +Time in milliseconds spent on message format conversions. +
+ +Message conversion rate + + +kafka.server:type=BrokerTopicMetrics,name={Produce|Fetch}MessageConversionsPerSec,topic=([-.\w]+) + + +Message format conversion rate, for Produce or Fetch requests, per topic. Omitting 'topic=(...)' will yield the all-topic rate. +
+ +Request Queue Size + + +kafka.network:type=RequestChannel,name=RequestQueueSize + + +Size of the request queue. +
+ +Byte out rate to clients + + +kafka.server:type=BrokerTopicMetrics,name=BytesOutPerSec,topic=([-.\w]+) + + +Byte out (to the clients) rate per topic. Omitting 'topic=(...)' will yield the all-topic rate. +
+ +Byte out rate to other brokers + + +kafka.server:type=BrokerTopicMetrics,name=ReplicationBytesOutPerSec + + +Byte out (to the other brokers) rate across all topics +
+ +Rejected byte rate + + +kafka.server:type=BrokerTopicMetrics,name=BytesRejectedPerSec,topic=([-.\w]+) + + +Rejected byte rate per topic, due to the record batch size being greater than max.message.bytes configuration. Omitting 'topic=(...)' will yield the all-topic rate. +
+ +Message validation failure rate due to no key specified for compacted topic + + +kafka.server:type=BrokerTopicMetrics,name=NoKeyCompactedTopicRecordsPerSec + + +0 +
+ +Message validation failure rate due to invalid magic number + + +kafka.server:type=BrokerTopicMetrics,name=InvalidMagicNumberRecordsPerSec + + +0 +
+ +Message validation failure rate due to incorrect crc checksum + + +kafka.server:type=BrokerTopicMetrics,name=InvalidMessageCrcRecordsPerSec + + +0 +
+ +Message validation failure rate due to non-continuous offset or sequence number in batch + + +kafka.server:type=BrokerTopicMetrics,name=InvalidOffsetOrSequenceRecordsPerSec + + +0 +
+ +Log flush rate and time + + +kafka.log:type=LogFlushStats,name=LogFlushRateAndTimeMs + + + +
+ +\# of offline log directories + + +kafka.log:type=LogManager,name=OfflineLogDirectoryCount + + +0 +
+ +Leader election rate + + +kafka.controller:type=ControllerStats,name=LeaderElectionRateAndTimeMs + + +non-zero when there are broker failures +
+ +Unclean leader election rate + + +kafka.controller:type=ControllerStats,name=UncleanLeaderElectionsPerSec + + +0 +
+ +Is controller active on broker + + +kafka.controller:type=KafkaController,name=ActiveControllerCount + + +only one broker in the cluster should have 1 +
+ +Pending topic deletes + + +kafka.controller:type=KafkaController,name=TopicsToDeleteCount + + + +
+ +Pending replica deletes + + +kafka.controller:type=KafkaController,name=ReplicasToDeleteCount + + + +
+ +Ineligible pending topic deletes + + +kafka.controller:type=KafkaController,name=TopicsIneligibleToDeleteCount + + + +
+ +Ineligible pending replica deletes + + +kafka.controller:type=KafkaController,name=ReplicasIneligibleToDeleteCount + + + +
+ +\# of under replicated partitions (|ISR| < |all replicas|) + + +kafka.server:type=ReplicaManager,name=UnderReplicatedPartitions + + +0 +
+ +\# of under minIsr partitions (|ISR| < min.insync.replicas) + + +kafka.server:type=ReplicaManager,name=UnderMinIsrPartitionCount + + +0 +
+ +\# of at minIsr partitions (|ISR| = min.insync.replicas) + + +kafka.server:type=ReplicaManager,name=AtMinIsrPartitionCount + + +0 +
+ +Producer Id counts + + +kafka.server:type=ReplicaManager,name=ProducerIdCount + + +Count of all producer ids created by transactional and idempotent producers in each replica on the broker +
+ +Partition counts + + +kafka.server:type=ReplicaManager,name=PartitionCount + + +mostly even across brokers +
+ +Offline Replica counts + + +kafka.server:type=ReplicaManager,name=OfflineReplicaCount + + +0 +
+ +Leader replica counts + + +kafka.server:type=ReplicaManager,name=LeaderCount + + +mostly even across brokers +
+ +ISR shrink rate + + +kafka.server:type=ReplicaManager,name=IsrShrinksPerSec + + +If a broker goes down, ISR for some of the partitions will shrink. When that broker is up again, ISR will be expanded once the replicas are fully caught up. Other than that, the expected value for both ISR shrink rate and expansion rate is 0. +
+ +ISR expansion rate + + +kafka.server:type=ReplicaManager,name=IsrExpandsPerSec + + +See above +
+ +Failed ISR update rate + + +kafka.server:type=ReplicaManager,name=FailedIsrUpdatesPerSec + + +0 +
+ +Max lag in messages btw follower and leader replicas + + +kafka.server:type=ReplicaFetcherManager,name=MaxLag,clientId=Replica + + +lag should be proportional to the maximum batch size of a produce request. +
+ +Lag in messages per follower replica + + +kafka.server:type=FetcherLagMetrics,name=ConsumerLag,clientId=([-.\w]+),topic=([-.\w]+),partition=([0-9]+) + + +lag should be proportional to the maximum batch size of a produce request. +
+ +Requests waiting in the producer purgatory + + +kafka.server:type=DelayedOperationPurgatory,name=PurgatorySize,delayedOperation=Produce + + +non-zero if ack=-1 is used +
+ +Requests waiting in the fetch purgatory + + +kafka.server:type=DelayedOperationPurgatory,name=PurgatorySize,delayedOperation=Fetch + + +size depends on fetch.wait.max.ms in the consumer +
+ +Request total time + + +kafka.network:type=RequestMetrics,name=TotalTimeMs,request={Produce|FetchConsumer|FetchFollower} + + +broken into queue, local, remote and response send time +
+ +Time the request waits in the request queue + + +kafka.network:type=RequestMetrics,name=RequestQueueTimeMs,request={Produce|FetchConsumer|FetchFollower} + + + +
+ +Time the request is processed at the leader + + +kafka.network:type=RequestMetrics,name=LocalTimeMs,request={Produce|FetchConsumer|FetchFollower} + + + +
+ +Time the request waits for the follower + + +kafka.network:type=RequestMetrics,name=RemoteTimeMs,request={Produce|FetchConsumer|FetchFollower} + + +non-zero for produce requests when ack=-1 +
+ +Time the request waits in the response queue + + +kafka.network:type=RequestMetrics,name=ResponseQueueTimeMs,request={Produce|FetchConsumer|FetchFollower} + + + +
+ +Time to send the response + + +kafka.network:type=RequestMetrics,name=ResponseSendTimeMs,request={Produce|FetchConsumer|FetchFollower} + + + +
+ +Number of messages the consumer lags behind the producer by. Published by the consumer, not broker. + + +kafka.consumer:type=consumer-fetch-manager-metrics,client-id={client-id} Attribute: records-lag-max + + + +
+ +The average fraction of time the network processors are idle + + +kafka.network:type=SocketServer,name=NetworkProcessorAvgIdlePercent + + +between 0 and 1, ideally > 0.3 +
+ +The number of connections disconnected on a processor due to a client not re-authenticating and then using the connection beyond its expiration time for anything other than re-authentication + + +kafka.server:type=socket-server-metrics,listener=[SASL_PLAINTEXT|SASL_SSL],networkProcessor=<#>,name=expired-connections-killed-count + + +ideally 0 when re-authentication is enabled, implying there are no longer any older, pre-2.2.0 clients connecting to this (listener, processor) combination +
+ +The total number of connections disconnected, across all processors, due to a client not re-authenticating and then using the connection beyond its expiration time for anything other than re-authentication + + +kafka.network:type=SocketServer,name=ExpiredConnectionsKilledCount + + +ideally 0 when re-authentication is enabled, implying there are no longer any older, pre-2.2.0 clients connecting to this broker +
+ +The average fraction of time the request handler threads are idle + + +kafka.server:type=KafkaRequestHandlerPool,name=RequestHandlerAvgIdlePercent + + +between 0 and 1, ideally > 0.3 +
+ +Bandwidth quota metrics per (user, client-id), user or client-id + + +kafka.server:type={Produce|Fetch},user=([-.\w]+),client-id=([-.\w]+) + + +Two attributes. throttle-time indicates the amount of time in ms the client was throttled. Ideally = 0. byte-rate indicates the data produce/consume rate of the client in bytes/sec. For (user, client-id) quotas, both user and client-id are specified. If per-client-id quota is applied to the client, user is not specified. If per-user quota is applied, client-id is not specified. +
+ +Request quota metrics per (user, client-id), user or client-id + + +kafka.server:type=Request,user=([-.\w]+),client-id=([-.\w]+) + + +Two attributes. throttle-time indicates the amount of time in ms the client was throttled. Ideally = 0. request-time indicates the percentage of time spent in broker network and I/O threads to process requests from client group. For (user, client-id) quotas, both user and client-id are specified. If per-client-id quota is applied to the client, user is not specified. If per-user quota is applied, client-id is not specified. +
+ +Requests exempt from throttling + + +kafka.server:type=Request + + +exempt-throttle-time indicates the percentage of time spent in broker network and I/O threads to process requests that are exempt from throttling. +
+ +ZooKeeper client request latency + + +kafka.server:type=ZooKeeperClientMetrics,name=ZooKeeperRequestLatencyMs + + +Latency in milliseconds for ZooKeeper requests from broker. +
+ +ZooKeeper connection status + + +kafka.server:type=SessionExpireListener,name=SessionState + + +Connection status of broker's ZooKeeper session which may be one of Disconnected|SyncConnected|AuthFailed|ConnectedReadOnly|SaslAuthenticated|Expired. +
+ +Max time to load group metadata + + +kafka.server:type=group-coordinator-metrics,name=partition-load-time-max + + +maximum time, in milliseconds, it took to load offsets and group metadata from the consumer offset partitions loaded in the last 30 seconds (including time spent waiting for the loading task to be scheduled) +
+ +Avg time to load group metadata + + +kafka.server:type=group-coordinator-metrics,name=partition-load-time-avg + + +average time, in milliseconds, it took to load offsets and group metadata from the consumer offset partitions loaded in the last 30 seconds (including time spent waiting for the loading task to be scheduled) +
+ +Max time to load transaction metadata + + +kafka.server:type=transaction-coordinator-metrics,name=partition-load-time-max + + +maximum time, in milliseconds, it took to load transaction metadata from the consumer offset partitions loaded in the last 30 seconds (including time spent waiting for the loading task to be scheduled) +
+ +Avg time to load transaction metadata + + +kafka.server:type=transaction-coordinator-metrics,name=partition-load-time-avg + + +average time, in milliseconds, it took to load transaction metadata from the consumer offset partitions loaded in the last 30 seconds (including time spent waiting for the loading task to be scheduled) +
+ +Rate of transactional verification errors + + +kafka.server:type=AddPartitionsToTxnManager,name=VerificationFailureRate + + +Rate of verifications that returned in failure either from the AddPartitionsToTxn API response or through errors in the AddPartitionsToTxnManager. In steady state 0, but transient errors are expected during rolls and reassignments of the transactional state partition. +
+ +Time to verify a transactional request + + +kafka.server:type=AddPartitionsToTxnManager,name=VerificationTimeMs + + +The amount of time queueing while a possible previous request is in-flight plus the round trip to the transaction coordinator to verify (or not verify) +
+ +Consumer Group Offset Count + + +kafka.server:type=GroupMetadataManager,name=NumOffsets + + +Total number of committed offsets for Consumer Groups +
+ +Consumer Group Count + + +kafka.server:type=GroupMetadataManager,name=NumGroups + + +Total number of Consumer Groups +
+ +Consumer Group Count, per State + + +kafka.server:type=GroupMetadataManager,name=NumGroups[PreparingRebalance,CompletingRebalance,Empty,Stable,Dead] + + +The number of Consumer Groups in each state: PreparingRebalance, CompletingRebalance, Empty, Stable, Dead +
+ +Number of reassigning partitions + + +kafka.server:type=ReplicaManager,name=ReassigningPartitions + + +The number of reassigning leader partitions on a broker. +
+ +Outgoing byte rate of reassignment traffic + + +kafka.server:type=BrokerTopicMetrics,name=ReassignmentBytesOutPerSec + + +0; non-zero when a partition reassignment is in progress. +
+ +Incoming byte rate of reassignment traffic + + +kafka.server:type=BrokerTopicMetrics,name=ReassignmentBytesInPerSec + + +0; non-zero when a partition reassignment is in progress. +
+ +Size of a partition on disk (in bytes) + + +kafka.log:type=Log,name=Size,topic=([-.\w]+),partition=([0-9]+) + + +The size of a partition on disk, measured in bytes. +
+ +Number of log segments in a partition + + +kafka.log:type=Log,name=NumLogSegments,topic=([-.\w]+),partition=([0-9]+) + + +The number of log segments in a partition. +
+ +First offset in a partition + + +kafka.log:type=Log,name=LogStartOffset,topic=([-.\w]+),partition=([0-9]+) + + +The first offset in a partition. +
+ +Last offset in a partition + + +kafka.log:type=Log,name=LogEndOffset,topic=([-.\w]+),partition=([0-9]+) + + +The last offset in a partition. +
+ +## Tiered Storage Monitoring + +The following set of metrics are available for monitoring of the tiered storage feature: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+ +Metric/Attribute name + + +Description + + +Mbean name +
+ +Remote Fetch Bytes Per Sec + + +Rate of bytes read from remote storage per topic. Omitting 'topic=(...)' will yield the all-topic rate + + +kafka.server:type=BrokerTopicMetrics,name=RemoteFetchBytesPerSec,topic=([-.\w]+) +
+ +Remote Fetch Requests Per Sec + + +Rate of read requests from remote storage per topic. Omitting 'topic=(...)' will yield the all-topic rate + + +kafka.server:type=BrokerTopicMetrics,name=RemoteFetchRequestsPerSec,topic=([-.\w]+) +
+ +Remote Fetch Errors Per Sec + + +Rate of read errors from remote storage per topic. Omitting 'topic=(...)' will yield the all-topic rate + + +kafka.server:type=BrokerTopicMetrics,name=RemoteFetchErrorsPerSec,topic=([-.\w]+) +
+ +Remote Copy Bytes Per Sec + + +Rate of bytes copied to remote storage per topic. Omitting 'topic=(...)' will yield the all-topic rate + + +kafka.server:type=BrokerTopicMetrics,name=RemoteCopyBytesPerSec,topic=([-.\w]+) +
+ +Remote Copy Requests Per Sec + + +Rate of write requests to remote storage per topic. Omitting 'topic=(...)' will yield the all-topic rate + + +kafka.server:type=BrokerTopicMetrics,name=RemoteCopyRequestsPerSec,topic=([-.\w]+) +
+ +Remote Copy Errors Per Sec + + +Rate of write errors from remote storage per topic. Omitting 'topic=(...)' will yield the all-topic rate + + +kafka.server:type=BrokerTopicMetrics,name=RemoteCopyErrorsPerSec,topic=([-.\w]+) +
+ +Remote Copy Lag Bytes + + +Bytes which are eligible for tiering, but are not in remote storage yet. Omitting 'topic=(...)' will yield the all-topic sum + + +kafka.server:type=BrokerTopicMetrics,name=RemoteCopyLagBytes,topic=([-.\w]+) +
+ +Remote Copy Lag Segments + + +Segments which are eligible for tiering, but are not in remote storage yet. Omitting 'topic=(...)' will yield the all-topic count + + +kafka.server:type=BrokerTopicMetrics,name=RemoteCopyLagSegments,topic=([-.\w]+) +
+ +Remote Delete Requests Per Sec + + +Rate of delete requests to remote storage per topic. Omitting 'topic=(...)' will yield the all-topic rate + + +kafka.server:type=BrokerTopicMetrics,name=RemoteDeleteRequestsPerSec,topic=([-.\w]+) +
+ +Remote Delete Errors Per Sec + + +Rate of delete errors from remote storage per topic. Omitting 'topic=(...)' will yield the all-topic rate + + +kafka.server:type=BrokerTopicMetrics,name=RemoteDeleteErrorsPerSec,topic=([-.\w]+) +
+ +Remote Delete Lag Bytes + + +Tiered bytes which are eligible for deletion, but have not been deleted yet. Omitting 'topic=(...)' will yield the all-topic sum + + +kafka.server:type=BrokerTopicMetrics,name=RemoteDeleteLagBytes,topic=([-.\w]+) +
+ +Remote Delete Lag Segments + + +Tiered segments which are eligible for deletion, but have not been deleted yet. Omitting 'topic=(...)' will yield the all-topic count + + +kafka.server:type=BrokerTopicMetrics,name=RemoteDeleteLagSegments,topic=([-.\w]+) +
+ +Build Remote Log Aux State Requests Per Sec + + +Rate of requests for rebuilding the auxiliary state from remote storage per topic. Omitting 'topic=(...)' will yield the all-topic rate + + +kafka.server:type=BrokerTopicMetrics,name=BuildRemoteLogAuxStateRequestsPerSec,topic=([-.\w]+) +
+ +Build Remote Log Aux State Errors Per Sec + + +Rate of errors for rebuilding the auxiliary state from remote storage per topic. Omitting 'topic=(...)' will yield the all-topic rate + + +kafka.server:type=BrokerTopicMetrics,name=BuildRemoteLogAuxStateErrorsPerSec,topic=([-.\w]+) +
+ +Remote Log Size Computation Time + + +The amount of time needed to compute the size of the remote log. Omitting 'topic=(...)' will yield the all-topic time + + +kafka.server:type=BrokerTopicMetrics,name=RemoteLogSizeComputationTime,topic=([-.\w]+) +
+ +Remote Log Size Bytes + + +The total size of a remote log in bytes. Omitting 'topic=(...)' will yield the all-topic sum + + +kafka.server:type=BrokerTopicMetrics,name=RemoteLogSizeBytes,topic=([-.\w]+) +
+ +Remote Log Metadata Count + + +The total number of metadata entries for remote storage. Omitting 'topic=(...)' will yield the all-topic count + + +kafka.server:type=BrokerTopicMetrics,name=RemoteLogMetadataCount,topic=([-.\w]+) +
+ +Delayed Remote Fetch Expires Per Sec + + +The number of expired remote fetches per second. Omitting 'topic=(...)' will yield the all-topic rate + + +kafka.server:type=DelayedRemoteFetchMetrics,name=ExpiresPerSec,topic=([-.\w]+) +
+ +RemoteLogReader Task Queue Size + + +Size of the queue holding remote storage read tasks + + +org.apache.kafka.storage.internals.log:type=RemoteStorageThreadPool,name=RemoteLogReaderTaskQueueSize +
+ +RemoteLogReader Avg Idle Percent + + +Average idle percent of thread pool for processing remote storage read tasks + + +org.apache.kafka.storage.internals.log:type=RemoteStorageThreadPool,name=RemoteLogReaderAvgIdlePercent +
+ +RemoteLogManager Tasks Avg Idle Percent + + +Average idle percent of thread pool for copying data to remote storage + + +kafka.log.remote:type=RemoteLogManager,name=RemoteLogManagerTasksAvgIdlePercent +
+ +## KRaft Monitoring Metrics + +The set of metrics that allow monitoring of the KRaft quorum and the metadata log. +Note that some exposed metrics depend on the role of the node as defined by `process.roles` + +### KRaft Quorum Monitoring Metrics + +These metrics are reported on both Controllers and Brokers in a KRaft Cluster + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+ +Metric/Attribute name + + +Description + + +Mbean name +
+ +Current State + + +The current state of this member; possible values are leader, candidate, voted, follower, unattached, observer. + + +kafka.server:type=raft-metrics,name=current-state +
+ +Current Leader + + +The current quorum leader's id; -1 indicates unknown. + + +kafka.server:type=raft-metrics,name=current-leader +
+ +Current Voted + + +The current voted leader's id; -1 indicates not voted for anyone. + + +kafka.server:type=raft-metrics,name=current-vote +
+ +Current Epoch + + +The current quorum epoch. + + +kafka.server:type=raft-metrics,name=current-epoch +
+ +High Watermark + + +The high watermark maintained on this member; -1 if it is unknown. + + +kafka.server:type=raft-metrics,name=high-watermark +
+ +Log End Offset + + +The current raft log end offset. + + +kafka.server:type=raft-metrics,name=log-end-offset +
+ +Number of Unknown Voter Connections + + +Number of unknown voters whose connection information is not cached. This value of this metric is always 0. + + +kafka.server:type=raft-metrics,name=number-unknown-voter-connections +
+ +Average Commit Latency + + +The average time in milliseconds to commit an entry in the raft log. + + +kafka.server:type=raft-metrics,name=commit-latency-avg +
+ +Maximum Commit Latency + + +The maximum time in milliseconds to commit an entry in the raft log. + + +kafka.server:type=raft-metrics,name=commit-latency-max +
+ +Average Election Latency + + +The average time in milliseconds spent on electing a new leader. + + +kafka.server:type=raft-metrics,name=election-latency-avg +
+ +Maximum Election Latency + + +The maximum time in milliseconds spent on electing a new leader. + + +kafka.server:type=raft-metrics,name=election-latency-max +
+ +Fetch Records Rate + + +The average number of records fetched from the leader of the raft quorum. + + +kafka.server:type=raft-metrics,name=fetch-records-rate +
+ +Append Records Rate + + +The average number of records appended per sec by the leader of the raft quorum. + + +kafka.server:type=raft-metrics,name=append-records-rate +
+ +Average Poll Idle Ratio + + +The average fraction of time the client's poll() is idle as opposed to waiting for the user code to process records. + + +kafka.server:type=raft-metrics,name=poll-idle-ratio-avg +
+ +Current Metadata Version + + +Outputs the feature level of the current effective metadata version. + + +kafka.server:type=MetadataLoader,name=CurrentMetadataVersion +
+ +Metadata Snapshot Load Count + + +The total number of times we have loaded a KRaft snapshot since the process was started. + + +kafka.server:type=MetadataLoader,name=HandleLoadSnapshotCount +
+ +Latest Metadata Snapshot Size + + +The total size in bytes of the latest snapshot that the node has generated. If none have been generated yet, this is the size of the latest snapshot that was loaded. If no snapshots have been generated or loaded, this is 0. + + +kafka.server:type=SnapshotEmitter,name=LatestSnapshotGeneratedBytes +
+ +Latest Metadata Snapshot Age + + +The interval in milliseconds since the latest snapshot that the node has generated. If none have been generated yet, this is approximately the time delta since the process was started. + + +kafka.server:type=SnapshotEmitter,name=LatestSnapshotGeneratedAgeMs +
+ +### KRaft Controller Monitoring Metrics + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+ +Metric/Attribute name + + +Description + + +Mbean name +
+ +Active Controller Count + + +The number of Active Controllers on this node. Valid values are '0' or '1'. + + +kafka.controller:type=KafkaController,name=ActiveControllerCount +
+ +Event Queue Time Ms + + +A Histogram of the time in milliseconds that requests spent waiting in the Controller Event Queue. + + +kafka.controller:type=ControllerEventManager,name=EventQueueTimeMs +
+ +Event Queue Processing Time Ms + + +A Histogram of the time in milliseconds that requests spent being processed in the Controller Event Queue. + + +kafka.controller:type=ControllerEventManager,name=EventQueueProcessingTimeMs +
+ +Fenced Broker Count + + +The number of fenced brokers as observed by this Controller. + + +kafka.controller:type=KafkaController,name=FencedBrokerCount +
+ +Active Broker Count + + +The number of active brokers as observed by this Controller. + + +kafka.controller:type=KafkaController,name=ActiveBrokerCount +
+ +Migrating ZK Broker Count + + +The number of brokers registered with the Controller that haven't yet migrated to KRaft mode. + + +kafka.controller:type=KafkaController,name=MigratingZkBrokerCount +
+ +ZK Migrating State + + + + + * 0 - NONE, cluster created in KRaft mode; + * 4 - ZK, Migration has not started, controller is a ZK controller; + * 2 - PRE_MIGRATION, the KRaft Controller is waiting for all ZK brokers to register in migration mode; + * 1 - MIGRATION, ZK metadata has been migrated, but some broker is still running in ZK mode; + * 3 - POST_MIGRATION, the cluster migration is complete; + + + + +kafka.controller:type=KafkaController,name=ZkMigrationState +
+ +Global Topic Count + + +The number of global topics as observed by this Controller. + + +kafka.controller:type=KafkaController,name=GlobalTopicCount +
+ +Global Partition Count + + +The number of global partitions as observed by this Controller. + + +kafka.controller:type=KafkaController,name=GlobalPartitionCount +
+ +Offline Partition Count + + +The number of offline topic partitions (non-internal) as observed by this Controller. + + +kafka.controller:type=KafkaController,name=OfflinePartitionsCount +
+ +Preferred Replica Imbalance Count + + +The count of topic partitions for which the leader is not the preferred leader. + + +kafka.controller:type=KafkaController,name=PreferredReplicaImbalanceCount +
+ +Metadata Error Count + + +The number of times this controller node has encountered an error during metadata log processing. + + +kafka.controller:type=KafkaController,name=MetadataErrorCount +
+ +Last Applied Record Offset + + +The offset of the last record from the cluster metadata partition that was applied by the Controller. + + +kafka.controller:type=KafkaController,name=LastAppliedRecordOffset +
+ +Last Committed Record Offset + + +The offset of the last record committed to this Controller. + + +kafka.controller:type=KafkaController,name=LastCommittedRecordOffset +
+ +Last Applied Record Timestamp + + +The timestamp of the last record from the cluster metadata partition that was applied by the Controller. + + +kafka.controller:type=KafkaController,name=LastAppliedRecordTimestamp +
+ +Last Applied Record Lag Ms + + +The difference between now and the timestamp of the last record from the cluster metadata partition that was applied by the controller. For active Controllers the value of this lag is always zero. + + +kafka.controller:type=KafkaController,name=LastAppliedRecordLagMs +
+ +ZooKeeper Write Behind Lag + + +The amount of lag in records that ZooKeeper is behind relative to the highest committed record in the metadata log. This metric will only be reported by the active KRaft controller. + + +kafka.controller:type=KafkaController,name=ZkWriteBehindLag +
+ +ZooKeeper Metadata Snapshot Write Time + + +The number of milliseconds the KRaft controller took reconciling a snapshot into ZooKeeper. + + +kafka.controller:type=KafkaController,name=ZkWriteSnapshotTimeMs +
+ +ZooKeeper Metadata Delta Write Time + + +The number of milliseconds the KRaft controller took writing a delta into ZK. + + +kafka.controller:type=KafkaController,name=ZkWriteDeltaTimeMs +
+ +Timed-out Broker Heartbeat Count + + +The number of broker heartbeats that timed out on this controller since the process was started. Note that only active controllers handle heartbeats, so only they will see increases in this metric. + + +kafka.controller:type=KafkaController,name=TimedOutBrokerHeartbeatCount +
+ +Number Of Operations Started In Event Queue + + +The total number of controller event queue operations that were started. This includes deferred operations. + + +kafka.controller:type=KafkaController,name=EventQueueOperationsStartedCount +
+ +Number of Operations Timed Out In Event Queue + + +The total number of controller event queue operations that timed out before they could be performed. + + +kafka.controller:type=KafkaController,name=EventQueueOperationsTimedOutCount +
+ +Number Of New Controller Elections + + +Counts the number of times this node has seen a new controller elected. A transition to the "no leader" state is not counted here. If the same controller as before becomes active, that still counts. + + +kafka.controller:type=KafkaController,name=NewActiveControllersCount +
+ +### KRaft Broker Monitoring Metrics + + + + + + + + + + + + + + + + + + + + + + + + + +
+ +Metric/Attribute name + + +Description + + +Mbean name +
+ +Last Applied Record Offset + + +The offset of the last record from the cluster metadata partition that was applied by the broker + + +kafka.server:type=broker-metadata-metrics,name=last-applied-record-offset +
+ +Last Applied Record Timestamp + + +The timestamp of the last record from the cluster metadata partition that was applied by the broker. + + +kafka.server:type=broker-metadata-metrics,name=last-applied-record-timestamp +
+ +Last Applied Record Lag Ms + + +The difference between now and the timestamp of the last record from the cluster metadata partition that was applied by the broker + + +kafka.server:type=broker-metadata-metrics,name=last-applied-record-lag-ms +
+ +Metadata Load Error Count + + +The number of errors encountered by the BrokerMetadataListener while loading the metadata log and generating a new MetadataDelta based on it. + + +kafka.server:type=broker-metadata-metrics,name=metadata-load-error-count +
+ +Metadata Apply Error Count + + +The number of errors encountered by the BrokerMetadataPublisher while applying a new MetadataImage based on the latest MetadataDelta. + + +kafka.server:type=broker-metadata-metrics,name=metadata-apply-error-count +
+ +## Common monitoring metrics for producer/consumer/connect/streams + +The following metrics are available on producer/consumer/connector/streams instances. For specific metrics, please see following sections. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+ +Metric/Attribute name + + +Description + + +Mbean name +
+ +connection-close-rate + + +Connections closed per second in the window. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +connection-close-total + + +Total connections closed in the window. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +connection-creation-rate + + +New connections established per second in the window. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +connection-creation-total + + +Total new connections established in the window. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +network-io-rate + + +The average number of network operations (reads or writes) on all connections per second. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +network-io-total + + +The total number of network operations (reads or writes) on all connections. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +outgoing-byte-rate + + +The average number of outgoing bytes sent per second to all servers. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +outgoing-byte-total + + +The total number of outgoing bytes sent to all servers. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +request-rate + + +The average number of requests sent per second. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +request-total + + +The total number of requests sent. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +request-size-avg + + +The average size of all requests in the window. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +request-size-max + + +The maximum size of any request sent in the window. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +incoming-byte-rate + + +Bytes/second read off all sockets. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +incoming-byte-total + + +Total bytes read off all sockets. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +response-rate + + +Responses received per second. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +response-total + + +Total responses received. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +select-rate + + +Number of times the I/O layer checked for new I/O to perform per second. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +select-total + + +Total number of times the I/O layer checked for new I/O to perform. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +io-wait-time-ns-avg + + +The average length of time the I/O thread spent waiting for a socket ready for reads or writes in nanoseconds. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +io-wait-time-ns-total + + +The total time the I/O thread spent waiting in nanoseconds. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +io-waittime-total + + +***Deprecated*** The total time the I/O thread spent waiting in nanoseconds. Replacement is `io-wait-time-ns-total`. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +io-wait-ratio + + +The fraction of time the I/O thread spent waiting. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +io-time-ns-avg + + +The average length of time for I/O per select call in nanoseconds. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +io-time-ns-total + + +The total time the I/O thread spent doing I/O in nanoseconds. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +iotime-total + + +***Deprecated*** The total time the I/O thread spent doing I/O in nanoseconds. Replacement is `io-time-ns-total`. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +io-ratio + + +The fraction of time the I/O thread spent doing I/O. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +connection-count + + +The current number of active connections. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +successful-authentication-rate + + +Connections per second that were successfully authenticated using SASL or SSL. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +successful-authentication-total + + +Total connections that were successfully authenticated using SASL or SSL. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +failed-authentication-rate + + +Connections per second that failed authentication. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +failed-authentication-total + + +Total connections that failed authentication. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +successful-reauthentication-rate + + +Connections per second that were successfully re-authenticated using SASL. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +successful-reauthentication-total + + +Total connections that were successfully re-authenticated using SASL. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +reauthentication-latency-max + + +The maximum latency in ms observed due to re-authentication. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +reauthentication-latency-avg + + +The average latency in ms observed due to re-authentication. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +failed-reauthentication-rate + + +Connections per second that failed re-authentication. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +failed-reauthentication-total + + +Total connections that failed re-authentication. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +successful-authentication-no-reauth-total + + +Total connections that were successfully authenticated by older, pre-2.2.0 SASL clients that do not support re-authentication. May only be non-zero. + + +kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+) +
+ +## Common Per-broker metrics for producer/consumer/connect/streams + +The following metrics are available on producer/consumer/connector/streams instances. For specific metrics, please see following sections. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+ +Metric/Attribute name + + +Description + + +Mbean name +
+ +outgoing-byte-rate + + +The average number of outgoing bytes sent per second for a node. + + +kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+) +
+ +outgoing-byte-total + + +The total number of outgoing bytes sent for a node. + + +kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+) +
+ +request-rate + + +The average number of requests sent per second for a node. + + +kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+) +
+ +request-total + + +The total number of requests sent for a node. + + +kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+) +
+ +request-size-avg + + +The average size of all requests in the window for a node. + + +kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+) +
+ +request-size-max + + +The maximum size of any request sent in the window for a node. + + +kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+) +
+ +incoming-byte-rate + + +The average number of bytes received per second for a node. + + +kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+) +
+ +incoming-byte-total + + +The total number of bytes received for a node. + + +kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+) +
+ +request-latency-avg + + +The average request latency in ms for a node. + + +kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+) +
+ +request-latency-max + + +The maximum request latency in ms for a node. + + +kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+) +
+ +response-rate + + +Responses received per second for a node. + + +kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+) +
+ +response-total + + +Total responses received for a node. + + +kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+) +
+ +## Producer monitoring + +The following metrics are available on producer instances. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+ +Metric/Attribute name + + +Description + + +Mbean name +
+ +waiting-threads + + +The number of user threads blocked waiting for buffer memory to enqueue their records. + + +kafka.producer:type=producer-metrics,client-id=([-.\w]+) +
+ +buffer-total-bytes + + +The maximum amount of buffer memory the client can use (whether or not it is currently used). + + +kafka.producer:type=producer-metrics,client-id=([-.\w]+) +
+ +buffer-available-bytes + + +The total amount of buffer memory that is not being used (either unallocated or in the free list). + + +kafka.producer:type=producer-metrics,client-id=([-.\w]+) +
+ +bufferpool-wait-time + + +The fraction of time an appender waits for space allocation. + + +kafka.producer:type=producer-metrics,client-id=([-.\w]+) +
+ +bufferpool-wait-time-total + + +***Deprecated*** The total time an appender waits for space allocation in nanoseconds. Replacement is `bufferpool-wait-time-ns-total` + + +kafka.producer:type=producer-metrics,client-id=([-.\w]+) +
+ +bufferpool-wait-time-ns-total + + +The total time an appender waits for space allocation in nanoseconds. + + +kafka.producer:type=producer-metrics,client-id=([-.\w]+) +
+ +flush-time-ns-total + + +The total time the Producer spent in Producer.flush in nanoseconds. + + +kafka.producer:type=producer-metrics,client-id=([-.\w]+) +
+ +txn-init-time-ns-total + + +The total time the Producer spent initializing transactions in nanoseconds (for EOS). + + +kafka.producer:type=producer-metrics,client-id=([-.\w]+) +
+ +txn-begin-time-ns-total + + +The total time the Producer spent in beginTransaction in nanoseconds (for EOS). + + +kafka.producer:type=producer-metrics,client-id=([-.\w]+) +
+ +txn-send-offsets-time-ns-total + + +The total time the Producer spent sending offsets to transactions in nanoseconds (for EOS). + + +kafka.producer:type=producer-metrics,client-id=([-.\w]+) +
+ +txn-commit-time-ns-total + + +The total time the Producer spent committing transactions in nanoseconds (for EOS). + + +kafka.producer:type=producer-metrics,client-id=([-.\w]+) +
+ +txn-abort-time-ns-total + + +The total time the Producer spent aborting transactions in nanoseconds (for EOS). + + +kafka.producer:type=producer-metrics,client-id=([-.\w]+) +
+ +### Producer Sender Metrics + +{{< include-html file="/static/37/generated/producer_metrics.html" >}} + +## Consumer monitoring + +The following metrics are available on consumer instances. + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+ +Metric/Attribute name + + +Description + + +Mbean name +
+ +time-between-poll-avg + + +The average delay between invocations of poll(). + + +kafka.consumer:type=consumer-metrics,client-id=([-.\w]+) +
+ +time-between-poll-max + + +The max delay between invocations of poll(). + + +kafka.consumer:type=consumer-metrics,client-id=([-.\w]+) +
+ +last-poll-seconds-ago + + +The number of seconds since the last poll() invocation. + + +kafka.consumer:type=consumer-metrics,client-id=([-.\w]+) +
+ +poll-idle-ratio-avg + + +The average fraction of time the consumer's poll() is idle as opposed to waiting for the user code to process records. + + +kafka.consumer:type=consumer-metrics,client-id=([-.\w]+) +
+ +committed-time-ns-total + + +The total time the Consumer spent in committed in nanoseconds. + + +kafka.consumer:type=consumer-metrics,client-id=([-.\w]+) +
+ +commit-sync-time-ns-total + + +The total time the Consumer spent committing offsets in nanoseconds (for AOS). + + +kafka.consumer:type=consumer-metrics,client-id=([-.\w]+) +
+ +### Consumer Group Metrics + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+ +Metric/Attribute name + + +Description + + +Mbean name +
+ +commit-latency-avg + + +The average time taken for a commit request + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +commit-latency-max + + +The max time taken for a commit request + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +commit-rate + + +The number of commit calls per second + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +commit-total + + +The total number of commit calls + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +assigned-partitions + + +The number of partitions currently assigned to this consumer + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +heartbeat-response-time-max + + +The max time taken to receive a response to a heartbeat request + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +heartbeat-rate + + +The average number of heartbeats per second + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +heartbeat-total + + +The total number of heartbeats + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +join-time-avg + + +The average time taken for a group rejoin + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +join-time-max + + +The max time taken for a group rejoin + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +join-rate + + +The number of group joins per second + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +join-total + + +The total number of group joins + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +sync-time-avg + + +The average time taken for a group sync + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +sync-time-max + + +The max time taken for a group sync + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +sync-rate + + +The number of group syncs per second + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +sync-total + + +The total number of group syncs + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +rebalance-latency-avg + + +The average time taken for a group rebalance + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +rebalance-latency-max + + +The max time taken for a group rebalance + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +rebalance-latency-total + + +The total time taken for group rebalances so far + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +rebalance-total + + +The total number of group rebalances participated + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +rebalance-rate-per-hour + + +The number of group rebalance participated per hour + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +failed-rebalance-total + + +The total number of failed group rebalances + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +failed-rebalance-rate-per-hour + + +The number of failed group rebalance event per hour + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +last-rebalance-seconds-ago + + +The number of seconds since the last rebalance event + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +last-heartbeat-seconds-ago + + +The number of seconds since the last controller heartbeat + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +partitions-revoked-latency-avg + + +The average time taken by the on-partitions-revoked rebalance listener callback + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +partitions-revoked-latency-max + + +The max time taken by the on-partitions-revoked rebalance listener callback + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +partitions-assigned-latency-avg + + +The average time taken by the on-partitions-assigned rebalance listener callback + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +partitions-assigned-latency-max + + +The max time taken by the on-partitions-assigned rebalance listener callback + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +partitions-lost-latency-avg + + +The average time taken by the on-partitions-lost rebalance listener callback + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +partitions-lost-latency-max + + +The max time taken by the on-partitions-lost rebalance listener callback + + +kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+) +
+ +### Consumer Fetch Metrics + +{{< include-html file="/static/37/generated/consumer_metrics.html" >}} + +## Connect Monitoring + +A Connect worker process contains all the producer and consumer metrics as well as metrics specific to Connect. The worker process itself has a number of metrics, while each connector and task have additional metrics. {{< include-html file="/static/37/generated/connect_metrics.html" >}} + +## Streams Monitoring + +A Kafka Streams instance contains all the producer and consumer metrics as well as additional metrics specific to Streams. The metrics have three recording levels: `info`, `debug`, and `trace`. + +Note that the metrics have a 4-layer hierarchy. At the top level there are client-level metrics for each started Kafka Streams client. Each client has stream threads, with their own metrics. Each stream thread has tasks, with their own metrics. Each task has a number of processor nodes, with their own metrics. Each task also has a number of state stores and record caches, all with their own metrics. + +Use the following configuration option to specify which metrics you want collected: + + + metrics.recording.level="info" + +### Client Metrics + +All of the following metrics have a recording level of `info`: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+ +Metric/Attribute name + + +Description + + +Mbean name +
+ +version + + +The version of the Kafka Streams client. + + +kafka.streams:type=stream-metrics,client-id=([-.\w]+) +
+ +commit-id + + +The version control commit ID of the Kafka Streams client. + + +kafka.streams:type=stream-metrics,client-id=([-.\w]+) +
+ +application-id + + +The application ID of the Kafka Streams client. + + +kafka.streams:type=stream-metrics,client-id=([-.\w]+) +
+ +topology-description + + +The description of the topology executed in the Kafka Streams client. + + +kafka.streams:type=stream-metrics,client-id=([-.\w]+) +
+ +state + + +The state of the Kafka Streams client. + + +kafka.streams:type=stream-metrics,client-id=([-.\w]+) +
+ +alive-stream-threads + + +The current number of alive stream threads that are running or participating in rebalance. + + +kafka.streams:type=stream-metrics,client-id=([-.\w]+) +
+ +failed-stream-threads + + +The number of failed stream threads since the start of the Kafka Streams client. + + +kafka.streams:type=stream-metrics,client-id=([-.\w]+) +
+ +### Thread Metrics + +All of the following metrics have a recording level of `info`: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+ +Metric/Attribute name + + +Description + + +Mbean name +
+ +commit-latency-avg + + +The average execution time in ms, for committing, across all running tasks of this thread. + + +kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+) +
+ +commit-latency-max + + +The maximum execution time in ms, for committing, across all running tasks of this thread. + + +kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+) +
+ +poll-latency-avg + + +The average execution time in ms, for consumer polling. + + +kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+) +
+ +poll-latency-max + + +The maximum execution time in ms, for consumer polling. + + +kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+) +
+ +process-latency-avg + + +The average execution time in ms, for processing. + + +kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+) +
+ +process-latency-max + + +The maximum execution time in ms, for processing. + + +kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+) +
+ +punctuate-latency-avg + + +The average execution time in ms, for punctuating. + + +kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+) +
+ +punctuate-latency-max + + +The maximum execution time in ms, for punctuating. + + +kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+) +
+ +commit-rate + + +The average number of commits per second. + + +kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+) +
+ +commit-total + + +The total number of commit calls. + + +kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+) +
+ +poll-rate + + +The average number of consumer poll calls per second. + + +kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+) +
+ +poll-total + + +The total number of consumer poll calls. + + +kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+) +
+ +process-rate + + +The average number of processed records per second. + + +kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+) +
+ +process-total + + +The total number of processed records. + + +kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+) +
+ +punctuate-rate + + +The average number of punctuate calls per second. + + +kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+) +
+ +punctuate-total + + +The total number of punctuate calls. + + +kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+) +
+ +task-created-rate + + +The average number of tasks created per second. + + +kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+) +
+ +task-created-total + + +The total number of tasks created. + + +kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+) +
+ +task-closed-rate + + +The average number of tasks closed per second. + + +kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+) +
+ +task-closed-total + + +The total number of tasks closed. + + +kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+) +
+ +blocked-time-ns-total + + +The total time the thread spent blocked on kafka. + + +kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+) +
+ +thread-start-time + + +The time that the thread was started. + + +kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+) +
+ +### Task Metrics + +All of the following metrics have a recording level of `debug`, except for the dropped-records-* and active-process-ratio metrics which have a recording level of `info`: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+ +Metric/Attribute name + + +Description + + +Mbean name +
+ +process-latency-avg + + +The average execution time in ns, for processing. + + +kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+) +
+ +process-latency-max + + +The maximum execution time in ns, for processing. + + +kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+) +
+ +process-rate + + +The average number of processed records per second across all source processor nodes of this task. + + +kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+) +
+ +process-total + + +The total number of processed records across all source processor nodes of this task. + + +kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+) +
+ +record-lateness-avg + + +The average observed lateness of records (stream time - record timestamp). + + +kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+) +
+ +record-lateness-max + + +The max observed lateness of records (stream time - record timestamp). + + +kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+) +
+ +enforced-processing-rate + + +The average number of enforced processings per second. + + +kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+) +
+ +enforced-processing-total + + +The total number enforced processings. + + +kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+) +
+ +dropped-records-rate + + +The average number of records dropped within this task. + + +kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+) +
+ +dropped-records-total + + +The total number of records dropped within this task. + + +kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+) +
+ +active-process-ratio + + +The fraction of time the stream thread spent on processing this task among all assigned active tasks. + + +kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+) +
+ +input-buffer-bytes-total + + +The total number of bytes accumulated by this task, + + +kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+) +
+ +cache-size-bytes-total + + +The cache size in bytes accumulated by this task. + + +kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+) +
+ +### Processor Node Metrics + +The following metrics are only available on certain types of nodes, i.e., the process-* metrics are only available for source processor nodes, the `suppression-emit-*` metrics are only available for suppression operation nodes, `emit-final-*` metrics are only available for windowed aggregations nodes, and the `record-e2e-latency-*` metrics are only available for source processor nodes and terminal nodes (nodes without successor nodes). All of the metrics have a recording level of `debug`, except for the `record-e2e-latency-*` metrics which have a recording level of `info`: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+ +Metric/Attribute name + + +Description + + +Mbean name +
+ +bytes-consumed-total + + +The total number of bytes consumed by a source processor node. + + +kafka.streams:type=stream-topic-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+),topic=([-.\w]+) +
+ +bytes-produced-total + + +The total number of bytes produced by a sink processor node. + + +kafka.streams:type=stream-topic-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+),topic=([-.\w]+) +
+ +process-rate + + +The average number of records processed by a source processor node per second. + + +kafka.streams:type=stream-processor-node-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+) +
+ +process-total + + +The total number of records processed by a source processor node per second. + + +kafka.streams:type=stream-processor-node-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+) +
+ +suppression-emit-rate + + +The rate at which records that have been emitted downstream from suppression operation nodes. + + +kafka.streams:type=stream-processor-node-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+) +
+ +suppression-emit-total + + +The total number of records that have been emitted downstream from suppression operation nodes. + + +kafka.streams:type=stream-processor-node-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+) +
+ +emit-final-latency-max + + +The max latency to emit final records when a record could be emitted. + + +kafka.streams:type=stream-processor-node-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+) +
+ +emit-final-latency-avg + + +The avg latency to emit final records when a record could be emitted. + + +kafka.streams:type=stream-processor-node-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+) +
+ +emit-final-records-rate + + +The rate of records emitted when records could be emitted. + + +kafka.streams:type=stream-processor-node-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+) +
+ +emit-final-records-total + + +The total number of records emitted. + + +kafka.streams:type=stream-processor-node-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+) +
+ +record-e2e-latency-avg + + +The average end-to-end latency of a record, measured by comparing the record timestamp with the system time when it has been fully processed by the node. + + +kafka.streams:type=stream-processor-node-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+) +
+ +record-e2e-latency-max + + +The maximum end-to-end latency of a record, measured by comparing the record timestamp with the system time when it has been fully processed by the node. + + +kafka.streams:type=stream-processor-node-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+) +
+ +record-e2e-latency-min + + +The minimum end-to-end latency of a record, measured by comparing the record timestamp with the system time when it has been fully processed by the node. + + +kafka.streams:type=stream-processor-node-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+) +
+ +records-consumed-total + + +The total number of records consumed by a source processor node. + + +kafka.streams:type=stream-topic-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+),topic=([-.\w]+) +
+ +records-produced-total + + +The total number of records produced by a sink processor node. + + +kafka.streams:type=stream-topic-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+),topic=([-.\w]+) +
+ +### State Store Metrics + +All of the following metrics have a recording level of `debug`, except for the record-e2e-latency-* metrics which have a recording level `trace`. Note that the `store-scope` value is specified in `StoreSupplier#metricsScope()` for user's customized state stores; for built-in state stores, currently we have: + + * `in-memory-state` + * `in-memory-lru-state` + * `in-memory-window-state` + * `in-memory-suppression` (for suppression buffers) + * `rocksdb-state` (for RocksDB backed key-value store) + * `rocksdb-window-state` (for RocksDB backed window store) + * `rocksdb-session-state` (for RocksDB backed session store) + +Metrics suppression-buffer-size-avg, suppression-buffer-size-max, suppression-buffer-count-avg, and suppression-buffer-count-max are only available for suppression buffers. All other metrics are not available for suppression buffers. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+ +Metric/Attribute name + + +Description + + +Mbean name +
+ +put-latency-avg + + +The average put execution time in ns. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +put-latency-max + + +The maximum put execution time in ns. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +put-if-absent-latency-avg + + +The average put-if-absent execution time in ns. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +put-if-absent-latency-max + + +The maximum put-if-absent execution time in ns. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +get-latency-avg + + +The average get execution time in ns. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +get-latency-max + + +The maximum get execution time in ns. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +delete-latency-avg + + +The average delete execution time in ns. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +delete-latency-max + + +The maximum delete execution time in ns. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +put-all-latency-avg + + +The average put-all execution time in ns. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +put-all-latency-max + + +The maximum put-all execution time in ns. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +all-latency-avg + + +The average all operation execution time in ns. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +all-latency-max + + +The maximum all operation execution time in ns. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +range-latency-avg + + +The average range execution time in ns. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +range-latency-max + + +The maximum range execution time in ns. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +flush-latency-avg + + +The average flush execution time in ns. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +flush-latency-max + + +The maximum flush execution time in ns. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +restore-latency-avg + + +The average restore execution time in ns. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +restore-latency-max + + +The maximum restore execution time in ns. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +put-rate + + +The average put rate for this store. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +put-if-absent-rate + + +The average put-if-absent rate for this store. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +get-rate + + +The average get rate for this store. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +delete-rate + + +The average delete rate for this store. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +put-all-rate + + +The average put-all rate for this store. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +all-rate + + +The average all operation rate for this store. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +range-rate + + +The average range rate for this store. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +flush-rate + + +The average flush rate for this store. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +restore-rate + + +The average restore rate for this store. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +suppression-buffer-size-avg + + +The average total size, in bytes, of the buffered data over the sampling window. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),in-memory-suppression-id=([-.\w]+) +
+ +suppression-buffer-size-max + + +The maximum total size, in bytes, of the buffered data over the sampling window. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),in-memory-suppression-id=([-.\w]+) +
+ +suppression-buffer-count-avg + + +The average number of records buffered over the sampling window. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),in-memory-suppression-id=([-.\w]+) +
+ +suppression-buffer-count-max + + +The maximum number of records buffered over the sampling window. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),in-memory-suppression-id=([-.\w]+) +
+ +record-e2e-latency-avg + + +The average end-to-end latency of a record, measured by comparing the record timestamp with the system time when it has been fully processed by the node. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +record-e2e-latency-max + + +The maximum end-to-end latency of a record, measured by comparing the record timestamp with the system time when it has been fully processed by the node. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +record-e2e-latency-min + + +The minimum end-to-end latency of a record, measured by comparing the record timestamp with the system time when it has been fully processed by the node. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +### RocksDB Metrics + +RocksDB metrics are grouped into statistics-based metrics and properties-based metrics. The former are recorded from statistics that a RocksDB state store collects whereas the latter are recorded from properties that RocksDB exposes. Statistics collected by RocksDB provide cumulative measurements over time, e.g. bytes written to the state store. Properties exposed by RocksDB provide current measurements, e.g., the amount of memory currently used. Note that the `store-scope` for built-in RocksDB state stores are currently the following: + + * `rocksdb-state` (for RocksDB backed key-value store) + * `rocksdb-window-state` (for RocksDB backed window store) + * `rocksdb-session-state` (for RocksDB backed session store) + +**RocksDB Statistics-based Metrics:** All of the following statistics-based metrics have a recording level of `debug` because collecting statistics in [RocksDB may have an impact on performance](https://github.com/facebook/rocksdb/wiki/Statistics#stats-level-and-performance-costs). Statistics-based metrics are collected every minute from the RocksDB state stores. If a state store consists of multiple RocksDB instances, as is the case for WindowStores and SessionStores, each metric reports an aggregation over the RocksDB instances of the state store. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+ +Metric/Attribute name + + +Description + + +Mbean name +
+ +bytes-written-rate + + +The average number of bytes written per second to the RocksDB state store. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +bytes-written-total + + +The total number of bytes written to the RocksDB state store. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +bytes-read-rate + + +The average number of bytes read per second from the RocksDB state store. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +bytes-read-total + + +The total number of bytes read from the RocksDB state store. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +memtable-bytes-flushed-rate + + +The average number of bytes flushed per second from the memtable to disk. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +memtable-bytes-flushed-total + + +The total number of bytes flushed from the memtable to disk. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +memtable-hit-ratio + + +The ratio of memtable hits relative to all lookups to the memtable. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +memtable-flush-time-avg + + +The average duration of memtable flushes to disc in ms. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +memtable-flush-time-min + + +The minimum duration of memtable flushes to disc in ms. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +memtable-flush-time-max + + +The maximum duration of memtable flushes to disc in ms. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +block-cache-data-hit-ratio + + +The ratio of block cache hits for data blocks relative to all lookups for data blocks to the block cache. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +block-cache-index-hit-ratio + + +The ratio of block cache hits for index blocks relative to all lookups for index blocks to the block cache. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +block-cache-filter-hit-ratio + + +The ratio of block cache hits for filter blocks relative to all lookups for filter blocks to the block cache. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +write-stall-duration-avg + + +The average duration of write stalls in ms. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +write-stall-duration-total + + +The total duration of write stalls in ms. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +bytes-read-compaction-rate + + +The average number of bytes read per second during compaction. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +bytes-written-compaction-rate + + +The average number of bytes written per second during compaction. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +compaction-time-avg + + +The average duration of disc compactions in ms. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +compaction-time-min + + +The minimum duration of disc compactions in ms. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +compaction-time-max + + +The maximum duration of disc compactions in ms. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +number-open-files + + +The number of current open files. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +number-file-errors-total + + +The total number of file errors occurred. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
**RocksDB Properties-based Metrics:** All of the following properties-based metrics have a recording level of `info` and are recorded when the metrics are accessed. If a state store consists of multiple RocksDB instances, as is the case for WindowStores and SessionStores, each metric reports the sum over all the RocksDB instances of the state store, except for the block cache metrics `block-cache-*`. The block cache metrics report the sum over all RocksDB instances if each instance uses its own block cache, and they report the recorded value from only one instance if a single block cache is shared among all instances. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+ +Metric/Attribute name + + +Description + + +Mbean name +
+ +num-immutable-mem-table + + +The number of immutable memtables that have not yet been flushed. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +cur-size-active-mem-table + + +The approximate size of the active memtable in bytes. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +cur-size-all-mem-tables + + +The approximate size of active and unflushed immutable memtables in bytes. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +size-all-mem-tables + + +The approximate size of active, unflushed immutable, and pinned immutable memtables in bytes. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +num-entries-active-mem-table + + +The number of entries in the active memtable. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +num-entries-imm-mem-tables + + +The number of entries in the unflushed immutable memtables. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +num-deletes-active-mem-table + + +The number of delete entries in the active memtable. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +num-deletes-imm-mem-tables + + +The number of delete entries in the unflushed immutable memtables. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +mem-table-flush-pending + + +This metric reports 1 if a memtable flush is pending, otherwise it reports 0. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +num-running-flushes + + +The number of currently running flushes. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +compaction-pending + + +This metric reports 1 if at least one compaction is pending, otherwise it reports 0. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +num-running-compactions + + +The number of currently running compactions. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +estimate-pending-compaction-bytes + + +The estimated total number of bytes a compaction needs to rewrite on disk to get all levels down to under target size (only valid for level compaction). + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +total-sst-files-size + + +The total size in bytes of all SST files. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +live-sst-files-size + + +The total size in bytes of all SST files that belong to the latest LSM tree. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +num-live-versions + + +Number of live versions of the LSM tree. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +block-cache-capacity + + +The capacity of the block cache in bytes. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +block-cache-usage + + +The memory size of the entries residing in block cache in bytes. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +block-cache-pinned-usage + + +The memory size for the entries being pinned in the block cache in bytes. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +estimate-num-keys + + +The estimated number of keys in the active and unflushed immutable memtables and storage. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +estimate-table-readers-mem + + +The estimated memory in bytes used for reading SST tables, excluding memory used in block cache. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +background-errors + + +The total number of background errors. + + +kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+) +
+ +### Record Cache Metrics + +All of the following metrics have a recording level of `debug`: + + + + + + + + + + + + + + + + +
+ +Metric/Attribute name + + +Description + + +Mbean name +
+ +hit-ratio-avg + + +The average cache hit ratio defined as the ratio of cache read hits over the total cache read requests. + + +kafka.streams:type=stream-record-cache-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),record-cache-id=([-.\w]+) +
+ +hit-ratio-min + + +The minimum cache hit ratio. + + +kafka.streams:type=stream-record-cache-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),record-cache-id=([-.\w]+) +
+ +hit-ratio-max + + +The maximum cache hit ratio. + + +kafka.streams:type=stream-record-cache-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),record-cache-id=([-.\w]+) +
+ +## Others + +We recommend monitoring GC time and other stats and various server stats such as CPU utilization, I/O service time, etc. On the client side, we recommend monitoring the message/byte rate (global and per topic), request rate/size/time, and on the consumer side, max lag in messages among all partitions and min fetch request rate. For a consumer to keep up, max lag needs to be less than a threshold and min fetch rate needs to be larger than 0. diff --git a/docs/operations/multi-tenancy.md b/docs/operations/multi-tenancy.md new file mode 100644 index 0000000000000..2ada0e58a759e --- /dev/null +++ b/docs/operations/multi-tenancy.md @@ -0,0 +1,116 @@ +--- +title: Multi-Tenancy +description: Multi-Tenancy +weight: 4 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +## Multi-Tenancy Overview + +As a highly scalable event streaming platform, Kafka is used by many users as their central nervous system, connecting in real-time a wide range of different systems and applications from various teams and lines of businesses. Such multi-tenant cluster environments command proper control and management to ensure the peaceful coexistence of these different needs. This section highlights features and best practices to set up such shared environments, which should help you operate clusters that meet SLAs/OLAs and that minimize potential collateral damage caused by "noisy neighbors". + +Multi-tenancy is a many-sided subject, including but not limited to: + + * Creating user spaces for tenants (sometimes called namespaces) + * Configuring topics with data retention policies and more + * Securing topics and clusters with encryption, authentication, and authorization + * Isolating tenants with quotas and rate limits + * Monitoring and metering + * Inter-cluster data sharing (cf. geo-replication) + + + +## Creating User Spaces (Namespaces) For Tenants With Topic Naming + +Kafka administrators operating a multi-tenant cluster typically need to define user spaces for each tenant. For the purpose of this section, "user spaces" are a collection of topics, which are grouped together under the management of a single entity or user. + +In Kafka, the main unit of data is the topic. Users can create and name each topic. They can also delete them, but it is not possible to rename a topic directly. Instead, to rename a topic, the user must create a new topic, move the messages from the original topic to the new, and then delete the original. With this in mind, it is recommended to define logical spaces, based on an hierarchical topic naming structure. This setup can then be combined with security features, such as prefixed ACLs, to isolate different spaces and tenants, while also minimizing the administrative overhead for securing the data in the cluster. + +These logical user spaces can be grouped in different ways, and the concrete choice depends on how your organization prefers to use your Kafka clusters. The most common groupings are as follows. + +_By team or organizational unit:_ Here, the team is the main aggregator. In an organization where teams are the main user of the Kafka infrastructure, this might be the best grouping. + +Example topic naming structure: + + * `...` +(e.g., "acme.infosec.telemetry.logins") + + + +_By project or product:_ Here, a team manages more than one project. Their credentials will be different for each project, so all the controls and settings will always be project related. + +Example topic naming structure: + + * `..` +(e.g., "mobility.payments.suspicious") + + + +Certain information should normally not be put in a topic name, such as information that is likely to change over time (e.g., the name of the intended consumer) or that is a technical detail or metadata that is available elsewhere (e.g., the topic's partition count and other configuration settings). + +To enforce a topic naming structure, several options are available: + + * Use prefix ACLs (cf. [KIP-290](https://cwiki.apache.org/confluence/display/KAFKA/KIP-290%3A+Support+for+Prefixed+ACLs)) to enforce a common prefix for topic names. For example, team A may only be permitted to create topics whose names start with `payments.teamA.`. + * Define a custom `CreateTopicPolicy` (cf. [KIP-108](https://cwiki.apache.org/confluence/display/KAFKA/KIP-108%3A+Create+Topic+Policy) and the setting create.topic.policy.class.name) to enforce strict naming patterns. These policies provide the most flexibility and can cover complex patterns and rules to match an organization's needs. + * Disable topic creation for normal users by denying it with an ACL, and then rely on an external process to create topics on behalf of users (e.g., scripting or your favorite automation toolkit). + * It may also be useful to disable the Kafka feature to auto-create topics on demand by setting `auto.create.topics.enable=false` in the broker configuration. Note that you should not rely solely on this option. + + + +## Configuring Topics: Data Retention And More + +Kafka's configuration is very flexible due to its fine granularity, and it supports a plethora of per-topic configuration settings to help administrators set up multi-tenant clusters. For example, administrators often need to define data retention policies to control how much and/or for how long data will be stored in a topic, with settings such as retention.bytes (size) and retention.ms (time). This limits storage consumption within the cluster, and helps complying with legal requirements such as GDPR. + +## Securing Clusters and Topics: Authentication, Authorization, Encryption + +Because the documentation has a dedicated chapter on security that applies to any Kafka deployment, this section focuses on additional considerations for multi-tenant environments. + +Security settings for Kafka fall into three main categories, which are similar to how administrators would secure other client-server data systems, like relational databases and traditional messaging systems. + + 1. **Encryption** of data transferred between Kafka brokers and Kafka clients, between brokers, between brokers and ZooKeeper nodes, and between brokers and other, optional tools. + 2. **Authentication** of connections from Kafka clients and applications to Kafka brokers, as well as connections from Kafka brokers to ZooKeeper nodes. + 3. **Authorization** of client operations such as creating, deleting, and altering the configuration of topics; writing events to or reading events from a topic; creating and deleting ACLs. Administrators can also define custom policies to put in place additional restrictions, such as a `CreateTopicPolicy` and `AlterConfigPolicy` (see [KIP-108](https://cwiki.apache.org/confluence/display/KAFKA/KIP-108%3A+Create+Topic+Policy) and the settings create.topic.policy.class.name, alter.config.policy.class.name). + + + +When securing a multi-tenant Kafka environment, the most common administrative task is the third category (authorization), i.e., managing the user/client permissions that grant or deny access to certain topics and thus to the data stored by users within a cluster. This task is performed predominantly through the setting of access control lists (ACLs). Here, administrators of multi-tenant environments in particular benefit from putting a hierarchical topic naming structure in place as described in a previous section, because they can conveniently control access to topics through prefixed ACLs (`--resource-pattern-type Prefixed`). This significantly minimizes the administrative overhead of securing topics in multi-tenant environments: administrators can make their own trade-offs between higher developer convenience (more lenient permissions, using fewer and broader ACLs) vs. tighter security (more stringent permissions, using more and narrower ACLs). + +In the following example, user Alice—a new member of ACME corporation's InfoSec team—is granted write permissions to all topics whose names start with "acme.infosec.", such as "acme.infosec.telemetry.logins" and "acme.infosec.syslogs.events". + + + # Grant permissions to user Alice + $ bin/kafka-acls.sh \ + --bootstrap-server broker1:9092 \ + --add --allow-principal User:Alice \ + --producer \ + --resource-pattern-type prefixed --topic acme.infosec. + + +You can similarly use this approach to isolate different customers on the same shared cluster. + +## Isolating Tenants: Quotas, Rate Limiting, Throttling + +Multi-tenant clusters should generally be configured with quotas, which protect against users (tenants) eating up too many cluster resources, such as when they attempt to write or read very high volumes of data, or create requests to brokers at an excessively high rate. This may cause network saturation, monopolize broker resources, and impact other clients—all of which you want to avoid in a shared environment. + +**Client quotas:** Kafka supports different types of (per-user principal) client quotas. Because a client's quotas apply irrespective of which topics the client is writing to or reading from, they are a convenient and effective tool to allocate resources in a multi-tenant cluster. Request rate quotas, for example, help to limit a user's impact on broker CPU usage by limiting the time a broker spends on the [request handling path](/protocol.html) for that user, after which throttling kicks in. In many situations, isolating users with request rate quotas has a bigger impact in multi-tenant clusters than setting incoming/outgoing network bandwidth quotas, because excessive broker CPU usage for processing requests reduces the effective bandwidth the broker can serve. Furthermore, administrators can also define quotas on topic operations—such as create, delete, and alter—to prevent Kafka clusters from being overwhelmed by highly concurrent topic operations (see [KIP-599](https://cwiki.apache.org/confluence/display/KAFKA/KIP-599%3A+Throttle+Create+Topic%2C+Create+Partition+and+Delete+Topic+Operations) and the quota type `controller_mutation_rate`). + +**Server quotas:** Kafka also supports different types of broker-side quotas. For example, administrators can set a limit on the rate with which the broker accepts new connections, set the maximum number of connections per broker, or set the maximum number of connections allowed from a specific IP address. + +For more information, please refer to the quota overview and how to set quotas. + +## Monitoring and Metering + +Monitoring is a broader subject that is covered elsewhere in the documentation. Administrators of any Kafka environment, but especially multi-tenant ones, should set up monitoring according to these instructions. Kafka supports a wide range of metrics, such as the rate of failed authentication attempts, request latency, consumer lag, total number of consumer groups, metrics on the quotas described in the previous section, and many more. + +For example, monitoring can be configured to track the size of topic-partitions (with the JMX metric `kafka.log.Log.Size.`), and thus the total size of data stored in a topic. You can then define alerts when tenants on shared clusters are getting close to using too much storage space. + +## Multi-Tenancy and Geo-Replication + +Kafka lets you share data across different clusters, which may be located in different geographical regions, data centers, and so on. Apart from use cases such as disaster recovery, this functionality is useful when a multi-tenant setup requires inter-cluster data sharing. See the section Geo-Replication (Cross-Cluster Data Mirroring) for more information. + +## Further considerations + +**Data contracts:** You may need to define data contracts between the producers and the consumers of data in a cluster, using event schemas. This ensures that events written to Kafka can always be read properly again, and prevents malformed or corrupt events being written. The best way to achieve this is to deploy a so-called schema registry alongside the cluster. (Kafka does not include a schema registry, but there are third-party implementations available.) A schema registry manages the event schemas and maps the schemas to topics, so that producers know which topics are accepting which types (schemas) of events, and consumers know how to read and parse events in a topic. Some registry implementations provide further functionality, such as schema evolution, storing a history of all schemas, and schema compatibility settings. diff --git a/docs/operations/tiered-storage.md b/docs/operations/tiered-storage.md new file mode 100644 index 0000000000000..e8fceb48c974e --- /dev/null +++ b/docs/operations/tiered-storage.md @@ -0,0 +1,139 @@ +--- +title: Tiered Storage +description: Tiered Storage +weight: 11 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +## Tiered Storage Overview + +Kafka data is mostly consumed in a streaming fashion using tail reads. Tail reads leverage OS's page cache to serve the data instead of disk reads. Older data is typically read from the disk for backfill or failure recovery purposes and is infrequent. + +In the tiered storage approach, Kafka cluster is configured with two tiers of storage - local and remote. The local tier is the same as the current Kafka that uses the local disks on the Kafka brokers to store the log segments. The new remote tier uses external storage systems, such as HDFS or S3, to store the completed log segments. Please check [KIP-405](https://cwiki.apache.org/confluence/display/KAFKA/KIP-405%3A+Kafka+Tiered+Storage) for more information. + +**Note: Tiered storage is considered as an early access feature, and is not recommended for use in production environments** + +## Configuration + +### Broker Configurations + +By default, Kafka server will not enable tiered storage feature. `remote.log.storage.system.enable` is the property to control whether to enable tiered storage functionality in a broker or not. Setting it to "true" enables this feature. + +`RemoteStorageManager` is an interface to provide the lifecycle of remote log segments and indexes. Kafka server doesn't provide out-of-the-box implementation of RemoteStorageManager. Configuring `remote.log.storage.manager.class.name` and `remote.log.storage.manager.class.path` to specify the implementation of RemoteStorageManager. + +`RemoteLogMetadataManager` is an interface to provide the lifecycle of metadata about remote log segments with strongly consistent semantics. By default, Kafka provides an implementation with storage as an internal topic. This implementation can be changed by configuring `remote.log.metadata.manager.class.name` and `remote.log.metadata.manager.class.path`. When adopting the default kafka internal topic based implementation, `remote.log.metadata.manager.listener.name` is a mandatory property to specify which listener the clients created by the default RemoteLogMetadataManager implementation. + +### Topic Configurations + +After correctly configuring broker side configurations for tiered storage feature, there are still configurations in topic level needed to be set. `remote.storage.enable` is the switch to determine if a topic wants to use tiered storage or not. By default it is set to false. After enabling `remote.storage.enable` property, the next thing to consider is the log retention. When tiered storage is enabled for a topic, there are 2 additional log retention configurations to set: + + * `local.retention.ms` + * `retention.ms` + * `local.retention.bytes` + * `retention.bytes` + + + +The configuration prefixed with `local` are to specify the time/size the "local" log file can accept before moving to remote storage, and then get deleted. If unset, The value in `retention.ms` and `retention.bytes` will be used. + +## Quick Start Example + +Apache Kafka doesn't provide an out-of-the-box RemoteStorageManager implementation. To have a preview of the tiered storage feature, the [LocalTieredStorage](https://github.com/apache/kafka/blob/trunk/storage/src/test/java/org/apache/kafka/server/log/remote/storage/LocalTieredStorage.java) implemented for integration test can be used, which will create a temporary directory in local storage to simulate the remote storage. + +To adopt the `LocalTieredStorage`, the test library needs to be built locally + + + # please checkout to the specific version tag you're using before building it + # ex: `git checkout 3.7.2` + ./gradlew clean :storage:testJar + +After build successfully, there should be a `kafka-storage-x.x.x-test.jar` file under `storage/build/libs`. Next, setting configurations in the broker side to enable tiered storage feature. + + + # Sample Zookeeper/Kraft broker server.properties listening on PLAINTEXT://:9092 + remote.log.storage.system.enable=true + + # Setting the listener for the clients in RemoteLogMetadataManager to talk to the brokers. + remote.log.metadata.manager.listener.name=PLAINTEXT + + # Please provide the implementation info for remoteStorageManager. + # This is the mandatory configuration for tiered storage. + # Here, we use the `LocalTieredStorage` built above. + remote.log.storage.manager.class.name=org.apache.kafka.server.log.remote.storage.LocalTieredStorage + remote.log.storage.manager.class.path=/PATH/TO/kafka-storage-3.7.2-test.jar + + # These 2 prefix are default values, but customizable + remote.log.storage.manager.impl.prefix=rsm.config. + remote.log.metadata.manager.impl.prefix=rlmm.config. + + # Configure the directory used for `LocalTieredStorage` + # Note, please make sure the brokers need to have access to this directory + rsm.config.dir=/tmp/kafka-remote-storage + + # This needs to be changed if number of brokers in the cluster is more than 1 + rlmm.config.remote.log.metadata.topic.replication.factor=1 + + # Try to speed up the log retention check interval for testing + log.retention.check.interval.ms=1000 + + +Following quick start guide to start up the kafka environment. Then, create a topic with tiered storage enabled with configs: + + + # remote.storage.enable=true -> enables tiered storage on the topic + # local.retention.ms=1000 -> The number of milliseconds to keep the local log segment before it gets deleted. + Note that a local log segment is eligible for deletion only after it gets uploaded to remote. + # retention.ms=3600000 -> when segments exceed this time, the segments in remote storage will be deleted + # segment.bytes=1048576 -> for test only, to speed up the log segment rolling interval + # file.delete.delay.ms=10000 -> for test only, to speed up the local-log segment file delete delay + + bin/kafka-topics.sh --create --topic tieredTopic --bootstrap-server localhost:9092 \ + --config remote.storage.enable=true --config local.retention.ms=1000 --config retention.ms=3600000 \ + --config segment.bytes=1048576 --config file.delete.delay.ms=1000 + + +Try to send messages to the `tieredTopic` topic to roll the log segment: + + + bin/kafka-producer-perf-test.sh --topic tieredTopic --num-records 1200 --record-size 1024 --throughput -1 --producer-props bootstrap.servers=localhost:9092 + + +Then, after the active segment is rolled, the old segment should be moved to the remote storage and get deleted. This can be verified by checking the remote log directory configured above. For example: + + + > ls /tmp/kafka-remote-storage/kafka-tiered-storage/tieredTopic-0-jF8s79t9SrG_PNqlwv7bAA + 00000000000000000000-knnxbs3FSRyKdPcSAOQC-w.index + 00000000000000000000-knnxbs3FSRyKdPcSAOQC-w.snapshot + 00000000000000000000-knnxbs3FSRyKdPcSAOQC-w.leader_epoch_checkpoint + 00000000000000000000-knnxbs3FSRyKdPcSAOQC-w.timeindex + 00000000000000000000-knnxbs3FSRyKdPcSAOQC-w.log + + +Lastly, we can try to consume some data from the beginning and print offset number, to make sure it will successfully fetch offset 0 from the remote storage. + + + bin/kafka-console-consumer.sh --topic tieredTopic --from-beginning --max-messages 1 --bootstrap-server localhost:9092 --property print.offset=true + +Please note, if you want to disable tiered storage at the cluster level, you should delete the tiered storage enabled topics explicitly. Attempting to disable tiered storage at the cluster level without deleting the topics using tiered storage will result in an exception during startup. + + + bin/kafka-topics.sh --delete --topic tieredTopic --bootstrap-server localhost:9092 + +After topics are deleted, you're safe to set `remote.log.storage.system.enable=false` in the broker configuration. + +## Limitations + +While the early access release of Tiered Storage offers the opportunity to try out this new feature, it is important to be aware of the following limitations: + + * No support for clusters with multiple log directories (i.e. JBOD feature) + * No support for compacted topics + * Cannot disable tiered storage at the topic level + * Deleting tiered storage enabled topics is required before disabling tiered storage at the broker level + * Admin actions related to tiered storage feature are only supported on clients from version 3.0 onwards + + + +For more information, please check [Tiered Storage Early Access Release Note](https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Tiered+Storage+Early+Access+Release+Notes). diff --git a/docs/operations/zookeeper.md b/docs/operations/zookeeper.md new file mode 100644 index 0000000000000..d22d1451215e4 --- /dev/null +++ b/docs/operations/zookeeper.md @@ -0,0 +1,43 @@ +--- +title: ZooKeeper +description: ZooKeeper +weight: 9 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +## Stable version + +The current stable branch is 3.8. Kafka is regularly updated to include the latest release in the 3.8 series. + +## ZooKeeper Deprecation + +With the release of Apache Kafka 3.5, Zookeeper is now marked deprecated. Removal of ZooKeeper is planned in the next major release of Apache Kafka (version 4.0), which is scheduled to happen no sooner than April 2024. During the deprecation phase, ZooKeeper is still supported for metadata management of Kafka clusters, but it is not recommended for new deployments. There is a small subset of features that remain to be implemented in KRaft see current missing features for more information. + +### Migration + +Users are recommended to begin planning for migration to KRaft and also begin testing to provide any feedback. Refer to ZooKeeper to KRaft Migration for details on how to perform a live migration from ZooKeeper to KRaft and current limitations. + +### x and ZooKeeper Support + +The final 3.x minor release, that supports ZooKeeper mode, will receive critical bug fixes and security fixes for 12 months after its release. + +### ZooKeeper and KRaft timeline + +For details and updates on tentative timelines for ZooKeeper removal and planned KRaft feature releases, refer to [KIP-833](https://cwiki.apache.org/confluence/display/KAFKA/KIP-833%3A+Mark+KRaft+as+Production+Ready). + +## Operationalizing ZooKeeper + +Operationally, we do the following for a healthy ZooKeeper installation: + + * Redundancy in the physical/hardware/network layout: try not to put them all in the same rack, decent (but don't go nuts) hardware, try to keep redundant power and network paths, etc. A typical ZooKeeper ensemble has 5 or 7 servers, which tolerates 2 and 3 servers down, respectively. If you have a small deployment, then using 3 servers is acceptable, but keep in mind that you'll only be able to tolerate 1 server down in this case. + * I/O segregation: if you do a lot of write type traffic you'll almost definitely want the transaction logs on a dedicated disk group. Writes to the transaction log are synchronous (but batched for performance), and consequently, concurrent writes can significantly affect performance. ZooKeeper snapshots can be one such a source of concurrent writes, and ideally should be written on a disk group separate from the transaction log. Snapshots are written to disk asynchronously, so it is typically ok to share with the operating system and message log files. You can configure a server to use a separate disk group with the dataLogDir parameter. + * Application segregation: Unless you really understand the application patterns of other apps that you want to install on the same box, it can be a good idea to run ZooKeeper in isolation (though this can be a balancing act with the capabilities of the hardware). + * Use care with virtualization: It can work, depending on your cluster layout and read/write patterns and SLAs, but the tiny overheads introduced by the virtualization layer can add up and throw off ZooKeeper, as it can be very time sensitive + * ZooKeeper configuration: It's java, make sure you give it 'enough' heap space (We usually run them with 3-5G, but that's mostly due to the data set size we have here). Unfortunately we don't have a good formula for it, but keep in mind that allowing for more ZooKeeper state means that snapshots can become large, and large snapshots affect recovery time. In fact, if the snapshot becomes too large (a few gigabytes), then you may need to increase the initLimit parameter to give enough time for servers to recover and join the ensemble. + * Monitoring: Both JMX and the 4 letter words (4lw) commands are very useful, they do overlap in some cases (and in those cases we prefer the 4 letter commands, they seem more predictable, or at the very least, they work better with the LI monitoring infrastructure) + * Don't overbuild the cluster: large clusters, especially in a write heavy usage pattern, means a lot of intracluster communication (quorums on the writes and subsequent cluster member updates), but don't underbuild it (and risk swamping the cluster). Having more servers adds to your read capacity. + +Overall, we try to keep the ZooKeeper system as small as will handle the load (plus standard growth capacity planning) and as simple as possible. We try not to do anything fancy with the configuration or application layout as compared to the official release as well as keep it as self contained as possible. For these reasons, we tend to skip the OS packaged versions, since it has a tendency to try to put things in the OS standard hierarchy, which can be 'messy', for want of a better way to word it. diff --git a/docs/ops.html b/docs/ops.html deleted file mode 100644 index 73a192a185056..0000000000000 --- a/docs/ops.html +++ /dev/null @@ -1,4339 +0,0 @@ - - - - -
diff --git a/docs/protocol.html b/docs/protocol.html deleted file mode 100644 index 29811a2cdbb4d..0000000000000 --- a/docs/protocol.html +++ /dev/null @@ -1,226 +0,0 @@ - - - - -
- -
-

Kafka protocol guide

- -

This document covers the wire protocol implemented in Kafka. It is meant to give a readable guide to the protocol that covers the available requests, their binary format, and the proper way to make use of them to implement a client. This document assumes you understand the basic design and terminology described here

- - - -

Preliminaries

- -
Network
- -

Kafka uses a binary protocol over TCP. The protocol defines all APIs as request response message pairs. All messages are size delimited and are made up of the following primitive types.

- -

The client initiates a socket connection and then writes a sequence of request messages and reads back the corresponding response message. No handshake is required on connection or disconnection. TCP is happier if you maintain persistent connections used for many requests to amortize the cost of the TCP handshake, but beyond this penalty connecting is pretty cheap.

- -

The client will likely need to maintain a connection to multiple brokers, as data is partitioned and the clients will need to talk to the server that has their data. However it should not generally be necessary to maintain multiple connections to a single broker from a single client instance (i.e. connection pooling).

- -

The server guarantees that on a single TCP connection, requests will be processed in the order they are sent and responses will return in that order as well. The broker's request processing allows only a single in-flight request per connection in order to guarantee this ordering. Note that clients can (and ideally should) use non-blocking IO to implement request pipelining and achieve higher throughput. i.e., clients can send requests even while awaiting responses for preceding requests since the outstanding requests will be buffered in the underlying OS socket buffer. All requests are initiated by the client, and result in a corresponding response message from the server except where noted.

- -

The server has a configurable maximum limit on request size and any request that exceeds this limit will result in the socket being disconnected.

- -
Partitioning and bootstrapping
- -

Kafka is a partitioned system so not all servers have the complete data set. Instead recall that topics are split into a pre-defined number of partitions, P, and each partition is replicated with some replication factor, N. Topic partitions themselves are just ordered "commit logs" numbered 0, 1, ..., P-1.

- -

All systems of this nature have the question of how a particular piece of data is assigned to a particular partition. Kafka clients directly control this assignment, the brokers themselves enforce no particular semantics of which messages should be published to a particular partition. Rather, to publish messages the client directly addresses messages to a particular partition, and when fetching messages, fetches from a particular partition. If two clients want to use the same partitioning scheme they must use the same method to compute the mapping of key to partition.

- -

These requests to publish or fetch data must be sent to the broker that is currently acting as the leader for a given partition. This condition is enforced by the broker, so a request for a particular partition to the wrong broker will result in an the NotLeaderForPartition error code (described below).

- -

How can the client find out which topics exist, what partitions they have, and which brokers currently host those partitions so that it can direct its requests to the right hosts? This information is dynamic, so you can't just configure each client with some static mapping file. Instead all Kafka brokers can answer a metadata request that describes the current state of the cluster: what topics there are, which partitions those topics have, which broker is the leader for those partitions, and the host and port information for these brokers.

- -

In other words, the client needs to somehow find one broker and that broker will tell the client about all the other brokers that exist and what partitions they host. This first broker may itself go down so the best practice for a client implementation is to take a list of two or three URLs to bootstrap from. The user can then choose to use a load balancer or just statically configure two or three of their Kafka hosts in the clients.

- -

The client does not need to keep polling to see if the cluster has changed; it can fetch metadata once when it is instantiated cache that metadata until it receives an error indicating that the metadata is out of date. This error can come in two forms: (1) a socket error indicating the client cannot communicate with a particular broker, (2) an error code in the response to a request indicating that this broker no longer hosts the partition for which data was requested.

-
    -
  1. Cycle through a list of "bootstrap" Kafka URLs until we find one we can connect to. Fetch cluster metadata.
  2. -
  3. Process fetch or produce requests, directing them to the appropriate broker based on the topic/partitions they send to or fetch from.
  4. -
  5. If we get an appropriate error, refresh the metadata and try again.
  6. -
- -
Partitioning Strategies
- -

As mentioned above the assignment of messages to partitions is something the producing client controls. That said, how should this functionality be exposed to the end-user?

- -

Partitioning really serves two purposes in Kafka:

-
    -
  1. It balances data and request load over brokers
  2. -
  3. It serves as a way to divvy up processing among consumer processes while allowing local state and preserving order within the partition. We call this semantic partitioning.
  4. -
- -

For a given use case you may care about only one of these or both.

- -

To accomplish simple load balancing a simple approach would be for the client to just round robin requests over all brokers. Another alternative, in an environment where there are many more producers than brokers, would be to have each client chose a single partition at random and publish to that. This later strategy will result in far fewer TCP connections.

- -

Semantic partitioning means using some key in the message to assign messages to partitions. For example if you were processing a click message stream you might want to partition the stream by the user id so that all data for a particular user would go to a single consumer. To accomplish this the client can take a key associated with the message and use some hash of this key to choose the partition to which to deliver the message.

- -
Batching
- -

Our APIs encourage batching small things together for efficiency. We have found this is a very significant performance win. Both our API to send messages and our API to fetch messages always work with a sequence of messages not a single message to encourage this. A clever client can make use of this and support an "asynchronous" mode in which it batches together messages sent individually and sends them in larger clumps. We go even further with this and allow the batching across multiple topics and partitions, so a produce request may contain data to append to many partitions and a fetch request may pull data from many partitions all at once.

- -

The client implementer can choose to ignore this and send everything one at a time if they like.

- -
Compatibility
- -

Kafka has a "bidirectional" client compatibility policy. In other words, new clients can talk to old servers, and old clients can talk to new servers. This allows users to upgrade either clients or servers without experiencing any downtime. - -

Since the Kafka protocol has changed over time, clients and servers need to agree on the schema of the message that they are sending over the wire. This is done through API versioning. - -

Before each request is sent, the client sends the API key and the API version. These two 16-bit numbers, when taken together, uniquely identify the schema of the message to follow. - -

The intention is that clients will support a range of API versions. When communicating with a particular broker, a given client should use the highest API version supported by both and indicate this version in their requests.

- -

The server will reject requests with a version it does not support, and will always respond to the client with exactly the protocol format it expects based on the version it included in its request. The intended upgrade path is that new features would first be rolled out on the server (with the older clients not making use of them) and then as newer clients are deployed these new features would gradually be taken advantage of.

- -

Note that KIP-482 tagged fields can be added to a request without incrementing the version number. This offers an additional way of evolving the message schema without breaking compatibility. Tagged fields do not take up any space when the field is not set. Therefore, if a field is rarely used, it is more efficient to make it a tagged field than to put it in the mandatory schema. However, tagged fields are ignored by recipients that don't know about them, which could pose a challenge if this is not the behavior that the sender wants. In such cases, a version bump may be more appropriate. - -

Retrieving Supported API versions
-

In order to work against multiple broker versions, clients need to know what versions of various APIs a - broker supports. The broker exposes this information since 0.10.0.0 as described in KIP-35. - Clients should use the supported API versions information to choose the highest API version supported by both client and broker. If no such version - exists, an error should be reported to the user.

-

The following sequence may be used by a client to obtain supported API versions from a broker.

-
    -
  1. Client sends ApiVersionsRequest to a broker after connection has been established with the broker. If SSL is enabled, - this happens after SSL connection has been established.
  2. -
  3. On receiving ApiVersionsRequest, a broker returns its full list of supported ApiKeys and - versions regardless of current authentication state (e.g., before SASL authentication on an SASL listener, do note that no - Kafka protocol requests may take place on an SSL listener before the SSL handshake is finished). If this is considered to - leak information about the broker version a workaround is to use SSL with client authentication which is performed at an - earlier stage of the connection where the ApiVersionRequest is not available. Also, note that broker versions older - than 0.10.0.0 do not support this API and will either ignore the request or close connection in response to the request.
  4. -
  5. If multiple versions of an API are supported by broker and client, clients are recommended to use the latest version supported - by the broker and itself.
  6. -
  7. Deprecation of a protocol version is done by marking an API version as deprecated in the protocol documentation.
  8. -
  9. Supported API versions obtained from a broker are only valid for the connection on which that information is obtained. - In the event of disconnection, the client should obtain the information from the broker again, as the broker might have been - upgraded/downgraded in the mean time.
  10. -
- -
SASL Authentication Sequence
-

The following sequence is used for SASL authentication: -

    -
  1. Kafka ApiVersionsRequest may be sent by the client to obtain the version ranges of requests supported by the broker. This is optional.
  2. -
  3. Kafka SaslHandshakeRequest containing the SASL mechanism for authentication is sent by the client. If the requested mechanism is not enabled - in the server, the server responds with the list of supported mechanisms and closes the client connection. If the mechanism is enabled - in the server, the server sends a successful response and continues with SASL authentication.
  4. -
  5. The actual SASL authentication is now performed. If SaslHandshakeRequest version is v0, a series of SASL client and server tokens corresponding to the mechanism are sent - as opaque packets without wrapping the messages with Kafka protocol headers. If SaslHandshakeRequest version is v1, the SaslAuthenticate - request/response are used, where the actual SASL tokens are wrapped in the Kafka protocol. The error code in the final message from the broker will indicate if authentication succeeded or failed.
  6. -
  7. If authentication succeeds, subsequent packets are handled as Kafka API requests. Otherwise, the client connection is closed.
  8. -
-

For interoperability with 0.9.0.x clients, the first packet received by the server is handled as a SASL/GSSAPI client token if it is not a valid -Kafka request. SASL/GSSAPI authentication is performed starting with this packet, skipping the first two steps above.

- - -

The Protocol

- -
Protocol Primitive Types
- -

The protocol is built out of the following primitive types.

- - -
Notes on reading the request format grammars
- -

The BNFs below give an exact context free grammar for the request and response binary format. The BNF is intentionally not compact in order to give human-readable name. As always in a BNF a sequence of productions indicates concatenation. When there are multiple possible productions these are separated with '|' and may be enclosed in parenthesis for grouping. The top-level definition is always given first and subsequent sub-parts are indented.

- -
Common Request and Response Structure
- -

All requests and responses originate from the following grammar which will be incrementally describe through the rest of this document:

- -
RequestOrResponse => Size (RequestMessage | ResponseMessage)
-  Size => int32
- - - - -
FieldDescription
message_sizeThe message_size field gives the size of the subsequent request or response message in bytes. The client can read requests by first reading this 4 byte size as an integer N, and then reading and parsing the subsequent N bytes of the request.
- -
Record Batch
-

A description of the record batch format can be found here.

- -

Constants

- -
Error Codes
-

We use numeric codes to indicate what problem occurred on the server. These can be translated by the client into exceptions or whatever the appropriate error handling mechanism in the client language. Here is a table of the error codes currently in use:

- - -
Api Keys
-

The following are the numeric codes that the ApiKey in the request can take for each of the below request types.

- - -

The Messages

- -

This section gives details on each of the individual API Messages, their usage, their binary format, and the meaning of their fields.

- - -

Some Common Philosophical Questions

- -

Some people have asked why we don't use HTTP. There are a number of reasons, the best is that client implementors can make use of some of the more advanced TCP features--the ability to multiplex requests, the ability to simultaneously poll many connections, etc. We have also found HTTP libraries in many languages to be surprisingly shabby.

- -

Others have asked if maybe we shouldn't support many different protocols. Prior experience with this was that it makes it very hard to add and test new features if they have to be ported across many protocol implementations. Our feeling is that most users don't really see multiple protocols as a feature, they just want a good reliable client in the language of their choice.

- -

Another question is why we don't adopt XMPP, STOMP, AMQP or an existing protocol. The answer to this varies by protocol, but in general the problem is that the protocol does determine large parts of the implementation and we couldn't do what we are doing if we didn't have control over the protocol. Our belief is that it is possible to do better than existing messaging systems have in providing a truly distributed messaging system, and to do this we need to build something that works differently.

- -

A final question is why we don't use a system like Protocol Buffers or Thrift to define our request messages. These packages excel at helping you to managing lots and lots of serialized messages. However we have only a few messages. Support across languages is somewhat spotty (depending on the package). Finally the mapping between binary log format and wire protocol is something we manage somewhat carefully and this would not be possible with these systems. Finally we prefer the style of versioning APIs explicitly and checking this to inferring new values as nulls as it allows more nuanced control of compatibility.

- - - - diff --git a/docs/quickstart.html b/docs/quickstart.html deleted file mode 100644 index e0c842793473b..0000000000000 --- a/docs/quickstart.html +++ /dev/null @@ -1,406 +0,0 @@ - - - - - - -
diff --git a/docs/security.html b/docs/security.html deleted file mode 100644 index 895f2b0b0a7a6..0000000000000 --- a/docs/security.html +++ /dev/null @@ -1,2436 +0,0 @@ - - - - -
diff --git a/docs/security/_index.md b/docs/security/_index.md new file mode 100644 index 0000000000000..734b80a622f17 --- /dev/null +++ b/docs/security/_index.md @@ -0,0 +1,10 @@ +--- +title: Security +description: +weight: 7 +tags: ['kafka', 'docs', 'security'] +aliases: +keywords: +type: docs +--- + diff --git a/docs/security/authentication-using-sasl.md b/docs/security/authentication-using-sasl.md new file mode 100644 index 0000000000000..faedcc03f3d6a --- /dev/null +++ b/docs/security/authentication-using-sasl.md @@ -0,0 +1,500 @@ +--- +title: Authentication using SASL +description: Authentication using SASL +weight: 4 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + + 1. #### JAAS configuration + +Kafka uses the Java Authentication and Authorization Service ([JAAS](https://docs.oracle.com/javase/8/docs/technotes/guides/security/jaas/JAASRefGuide.html)) for SASL configuration. + + 1. ##### JAAS configuration for Kafka brokers + +`KafkaServer` is the section name in the JAAS file used by each KafkaServer/Broker. This section provides SASL configuration options for the broker including any SASL client connections made by the broker for inter-broker communication. If multiple listeners are configured to use SASL, the section name may be prefixed with the listener name in lower-case followed by a period, e.g. `sasl_ssl.KafkaServer`. + +`Client` section is used to authenticate a SASL connection with zookeeper. It also allows the brokers to set SASL ACL on zookeeper nodes which locks these nodes down so that only the brokers can modify it. It is necessary to have the same principal name across all brokers. If you want to use a section name other than Client, set the system property `zookeeper.sasl.clientconfig` to the appropriate name (_e.g._ , `-Dzookeeper.sasl.clientconfig=ZkClient`). + +ZooKeeper uses "zookeeper" as the service name by default. If you want to change this, set the system property `zookeeper.sasl.client.username` to the appropriate name (_e.g._ , `-Dzookeeper.sasl.client.username=zk`). + +Brokers may also configure JAAS using the broker configuration property `sasl.jaas.config`. The property name must be prefixed with the listener prefix including the SASL mechanism, i.e. `listener.name.{listenerName}.{saslMechanism}.sasl.jaas.config`. Only one login module may be specified in the config value. If multiple mechanisms are configured on a listener, configs must be provided for each mechanism using the listener and mechanism prefix. For example, + + listener.name.sasl_ssl.scram-sha-256.sasl.jaas.config=org.apache.kafka.common.security.scram.ScramLoginModule required \ + username="admin" \ + password="admin-secret"; + listener.name.sasl_ssl.plain.sasl.jaas.config=org.apache.kafka.common.security.plain.PlainLoginModule required \ + username="admin" \ + password="admin-secret" \ + user_admin="admin-secret" \ + user_alice="alice-secret"; + +If JAAS configuration is defined at different levels, the order of precedence used is: + * Broker configuration property `listener.name.{listenerName}.{saslMechanism}.sasl.jaas.config` + * `{listenerName}.KafkaServer` section of static JAAS configuration + * `KafkaServer` section of static JAAS configuration +Note that ZooKeeper JAAS config may only be configured using static JAAS configuration. + +See GSSAPI (Kerberos), PLAIN, SCRAM or OAUTHBEARER for example broker configurations. + + 2. ##### JAAS configuration for Kafka clients + +Clients may configure JAAS using the client configuration property sasl.jaas.config or using the static JAAS config file similar to brokers. + + 1. ###### JAAS configuration using client configuration property + +Clients may specify JAAS configuration as a producer or consumer property without creating a physical configuration file. This mode also enables different producers and consumers within the same JVM to use different credentials by specifying different properties for each client. If both static JAAS configuration system property `java.security.auth.login.config` and client property `sasl.jaas.config` are specified, the client property will be used. + +See GSSAPI (Kerberos), PLAIN, SCRAM or OAUTHBEARER for example configurations. + + 2. ###### JAAS configuration using static config file + +To configure SASL authentication on the clients using static JAAS config file: + 1. Add a JAAS config file with a client login section named `KafkaClient`. Configure a login module in `KafkaClient` for the selected mechanism as described in the examples for setting up GSSAPI (Kerberos), PLAIN, SCRAM or OAUTHBEARER. For example, GSSAPI credentials may be configured as: + + KafkaClient { + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + storeKey=true + keyTab="/etc/security/keytabs/kafka_client.keytab" + principal="kafka-client-1@EXAMPLE.COM"; + }; + + 2. Pass the JAAS config file location as JVM parameter to each client JVM. For example: + + -Djava.security.auth.login.config=/etc/kafka/kafka_client_jaas.conf + + 2. #### SASL configuration + +SASL may be used with PLAINTEXT or SSL as the transport layer using the security protocol SASL_PLAINTEXT or SASL_SSL respectively. If SASL_SSL is used, then SSL must also be configured. + + 1. ##### SASL mechanisms + +Kafka supports the following SASL mechanisms: + * GSSAPI (Kerberos) + * PLAIN + * SCRAM-SHA-256 + * SCRAM-SHA-512 + * OAUTHBEARER + 2. ##### SASL configuration for Kafka brokers + + 1. Configure a SASL port in server.properties, by adding at least one of SASL_PLAINTEXT or SASL_SSL to the _listeners_ parameter, which contains one or more comma-separated values: + + listeners=SASL_PLAINTEXT://host.name:port + +If you are only configuring a SASL port (or if you want the Kafka brokers to authenticate each other using SASL) then make sure you set the same SASL protocol for inter-broker communication: + + security.inter.broker.protocol=SASL_PLAINTEXT (or SASL_SSL) + + 2. Select one or more supported mechanisms to enable in the broker and follow the steps to configure SASL for the mechanism. To enable multiple mechanisms in the broker, follow the steps here. + 3. ##### SASL configuration for Kafka clients + +SASL authentication is only supported for the new Java Kafka producer and consumer, the older API is not supported. + +To configure SASL authentication on the clients, select a SASL mechanism that is enabled in the broker for client authentication and follow the steps to configure SASL for the selected mechanism. + +Note: When establishing connections to brokers via SASL, clients may perform a reverse DNS lookup of the broker address. Due to how the JRE implements reverse DNS lookups, clients may observe slow SASL handshakes if fully qualified domain names are not used, for both the client's `bootstrap.servers` and a broker's `advertised.listeners`. + + 3. #### Authentication using SASL/Kerberos + + 1. ##### Prerequisites + + 1. **Kerberos** +If your organization is already using a Kerberos server (for example, by using Active Directory), there is no need to install a new server just for Kafka. Otherwise you will need to install one, your Linux vendor likely has packages for Kerberos and a short guide on how to install and configure it ([Ubuntu](https://help.ubuntu.com/community/Kerberos), [Redhat](https://access.redhat.com/en-US/Red_Hat_Enterprise_Linux/6/html/Managing_Smart_Cards/installing-kerberos.html)). Note that if you are using Oracle Java, you will need to download JCE policy files for your Java version and copy them to $JAVA_HOME/jre/lib/security. + 2. **Create Kerberos Principals** +If you are using the organization's Kerberos or Active Directory server, ask your Kerberos administrator for a principal for each Kafka broker in your cluster and for every operating system user that will access Kafka with Kerberos authentication (via clients and tools). +If you have installed your own Kerberos, you will need to create these principals yourself using the following commands: + + > sudo /usr/sbin/kadmin.local -q 'addprinc -randkey kafka/{hostname}@{REALM}' + > sudo /usr/sbin/kadmin.local -q "ktadd -k /etc/security/keytabs/{keytabname}.keytab kafka/{hostname}@{REALM}" + + 3. **Make sure all hosts can be reachable using hostnames** \- it is a Kerberos requirement that all your hosts can be resolved with their FQDNs. + 2. ##### Configuring Kafka Brokers + + 1. Add a suitably modified JAAS file similar to the one below to each Kafka broker's config directory, let's call it kafka_server_jaas.conf for this example (note that each broker should have its own keytab): + + KafkaServer { + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + storeKey=true + keyTab="/etc/security/keytabs/kafka_server.keytab" + principal="kafka/kafka1.hostname.com@EXAMPLE.COM"; + }; + + // Zookeeper client authentication + Client { + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + storeKey=true + keyTab="/etc/security/keytabs/kafka_server.keytab" + principal="kafka/kafka1.hostname.com@EXAMPLE.COM"; + }; + +`KafkaServer` section in the JAAS file tells the broker which principal to use and the location of the keytab where this principal is stored. It allows the broker to login using the keytab specified in this section. See notes for more details on Zookeeper SASL configuration. + 2. Pass the JAAS and optionally the krb5 file locations as JVM parameters to each Kafka broker (see [here](https://docs.oracle.com/javase/8/docs/technotes/guides/security/jgss/tutorials/KerberosReq.html) for more details): + + -Djava.security.krb5.conf=/etc/kafka/krb5.conf + -Djava.security.auth.login.config=/etc/kafka/kafka_server_jaas.conf + + 3. Make sure the keytabs configured in the JAAS file are readable by the operating system user who is starting kafka broker. + 4. Configure SASL port and SASL mechanisms in server.properties as described here. For example: + + listeners=SASL_PLAINTEXT://host.name:port + security.inter.broker.protocol=SASL_PLAINTEXT + sasl.mechanism.inter.broker.protocol=GSSAPI + sasl.enabled.mechanisms=GSSAPI + +We must also configure the service name in server.properties, which should match the principal name of the kafka brokers. In the above example, principal is "kafka/kafka1.hostname.com@EXAMPLE.com", so: + + sasl.kerberos.service.name=kafka + + 3. ##### Configuring Kafka Clients + +To configure SASL authentication on the clients: + 1. Clients (producers, consumers, connect workers, etc) will authenticate to the cluster with their own principal (usually with the same name as the user running the client), so obtain or create these principals as needed. Then configure the JAAS configuration property for each client. Different clients within a JVM may run as different users by specifying different principals. The property `sasl.jaas.config` in producer.properties or consumer.properties describes how clients like producer and consumer can connect to the Kafka Broker. The following is an example configuration for a client using a keytab (recommended for long-running processes): + + sasl.jaas.config=com.sun.security.auth.module.Krb5LoginModule required \ + useKeyTab=true \ + storeKey=true \ + keyTab="/etc/security/keytabs/kafka_client.keytab" \ + principal="kafka-client-1@EXAMPLE.COM"; + +For command-line utilities like kafka-console-consumer or kafka-console-producer, kinit can be used along with "useTicketCache=true" as in: + + sasl.jaas.config=com.sun.security.auth.module.Krb5LoginModule required \ + useTicketCache=true; + +JAAS configuration for clients may alternatively be specified as a JVM parameter similar to brokers as described here. Clients use the login section named `KafkaClient`. This option allows only one user for all client connections from a JVM. + 2. Make sure the keytabs configured in the JAAS configuration are readable by the operating system user who is starting kafka client. + 3. Optionally pass the krb5 file locations as JVM parameters to each client JVM (see [here](https://docs.oracle.com/javase/8/docs/technotes/guides/security/jgss/tutorials/KerberosReq.html) for more details): + + -Djava.security.krb5.conf=/etc/kafka/krb5.conf + + 4. Configure the following properties in producer.properties or consumer.properties: + + security.protocol=SASL_PLAINTEXT (or SASL_SSL) + sasl.mechanism=GSSAPI + sasl.kerberos.service.name=kafka + + 4. #### Authentication using SASL/PLAIN + +SASL/PLAIN is a simple username/password authentication mechanism that is typically used with TLS for encryption to implement secure authentication. Kafka supports a default implementation for SASL/PLAIN which can be extended for production use as described here. + +Under the default implementation of `principal.builder.class`, the username is used as the authenticated `Principal` for configuration of ACLs etc. + 1. ##### Configuring Kafka Brokers + + 1. Add a suitably modified JAAS file similar to the one below to each Kafka broker's config directory, let's call it kafka_server_jaas.conf for this example: + + KafkaServer { + org.apache.kafka.common.security.plain.PlainLoginModule required + username="admin" + password="admin-secret" + user_admin="admin-secret" + user_alice="alice-secret"; + }; + +This configuration defines two users (_admin_ and _alice_). The properties `username` and `password` in the `KafkaServer` section are used by the broker to initiate connections to other brokers. In this example, _admin_ is the user for inter-broker communication. The set of properties `user__userName_` defines the passwords for all users that connect to the broker and the broker validates all client connections including those from other brokers using these properties. + 2. Pass the JAAS config file location as JVM parameter to each Kafka broker: + + -Djava.security.auth.login.config=/etc/kafka/kafka_server_jaas.conf + + 3. Configure SASL port and SASL mechanisms in server.properties as described here. For example: + + listeners=SASL_SSL://host.name:port + security.inter.broker.protocol=SASL_SSL + sasl.mechanism.inter.broker.protocol=PLAIN + sasl.enabled.mechanisms=PLAIN + + 2. ##### Configuring Kafka Clients + +To configure SASL authentication on the clients: + 1. Configure the JAAS configuration property for each client in producer.properties or consumer.properties. The login module describes how the clients like producer and consumer can connect to the Kafka Broker. The following is an example configuration for a client for the PLAIN mechanism: + + sasl.jaas.config=org.apache.kafka.common.security.plain.PlainLoginModule required \ + username="alice" \ + password="alice-secret"; + +The options `username` and `password` are used by clients to configure the user for client connections. In this example, clients connect to the broker as user _alice_. Different clients within a JVM may connect as different users by specifying different user names and passwords in `sasl.jaas.config`. + +JAAS configuration for clients may alternatively be specified as a JVM parameter similar to brokers as described here. Clients use the login section named `KafkaClient`. This option allows only one user for all client connections from a JVM. + + 2. Configure the following properties in producer.properties or consumer.properties: + + security.protocol=SASL_SSL + sasl.mechanism=PLAIN + + 3. ##### Use of SASL/PLAIN in production + + * SASL/PLAIN should be used only with SSL as transport layer to ensure that clear passwords are not transmitted on the wire without encryption. + * The default implementation of SASL/PLAIN in Kafka specifies usernames and passwords in the JAAS configuration file as shown here. From Kafka version 2.0 onwards, you can avoid storing clear passwords on disk by configuring your own callback handlers that obtain username and password from an external source using the configuration options `sasl.server.callback.handler.class` and `sasl.client.callback.handler.class`. + * In production systems, external authentication servers may implement password authentication. From Kafka version 2.0 onwards, you can plug in your own callback handlers that use external authentication servers for password verification by configuring `sasl.server.callback.handler.class`. + 5. #### Authentication using SASL/SCRAM + +Salted Challenge Response Authentication Mechanism (SCRAM) is a family of SASL mechanisms that addresses the security concerns with traditional mechanisms that perform username/password authentication like PLAIN and DIGEST-MD5. The mechanism is defined in [RFC 5802](https://tools.ietf.org/html/rfc5802). Kafka supports [SCRAM-SHA-256](https://tools.ietf.org/html/rfc7677) and SCRAM-SHA-512 which can be used with TLS to perform secure authentication. Under the default implementation of `principal.builder.class`, the username is used as the authenticated `Principal` for configuration of ACLs etc. The default SCRAM implementation in Kafka stores SCRAM credentials in Zookeeper and is suitable for use in Kafka installations where Zookeeper is on a private network. Refer to Security Considerations for more details. + + 1. ##### Creating SCRAM Credentials + +The SCRAM implementation in Kafka uses Zookeeper as credential store. Credentials can be created in Zookeeper using `kafka-configs.sh`. For each SCRAM mechanism enabled, credentials must be created by adding a config with the mechanism name. Credentials for inter-broker communication must be created before Kafka brokers are started. Client credentials may be created and updated dynamically and updated credentials will be used to authenticate new connections. + +Create SCRAM credentials for user _alice_ with password _alice-secret_ : + + > bin/kafka-configs.sh --zookeeper localhost:2182 --zk-tls-config-file zk_tls_config.properties --alter --add-config 'SCRAM-SHA-256=[iterations=8192,password=alice-secret],SCRAM-SHA-512=[password=alice-secret]' --entity-type users --entity-name alice + +The default iteration count of 4096 is used if iterations are not specified. A random salt is created and the SCRAM identity consisting of salt, iterations, StoredKey and ServerKey are stored in Zookeeper. See [RFC 5802](https://tools.ietf.org/html/rfc5802) for details on SCRAM identity and the individual fields. + +The following examples also require a user _admin_ for inter-broker communication which can be created using: + + > bin/kafka-configs.sh --zookeeper localhost:2182 --zk-tls-config-file zk_tls_config.properties --alter --add-config 'SCRAM-SHA-256=[password=admin-secret],SCRAM-SHA-512=[password=admin-secret]' --entity-type users --entity-name admin + +Existing credentials may be listed using the _\--describe_ option: + + > bin/kafka-configs.sh --zookeeper localhost:2182 --zk-tls-config-file zk_tls_config.properties --describe --entity-type users --entity-name alice + +Credentials may be deleted for one or more SCRAM mechanisms using the _\--alter --delete-config_ option: + + > bin/kafka-configs.sh --zookeeper localhost:2182 --zk-tls-config-file zk_tls_config.properties --alter --delete-config 'SCRAM-SHA-512' --entity-type users --entity-name alice + + 2. ##### Configuring Kafka Brokers + + 1. Add a suitably modified JAAS file similar to the one below to each Kafka broker's config directory, let's call it kafka_server_jaas.conf for this example: + + KafkaServer { + org.apache.kafka.common.security.scram.ScramLoginModule required + username="admin" + password="admin-secret"; + }; + +The properties `username` and `password` in the `KafkaServer` section are used by the broker to initiate connections to other brokers. In this example, _admin_ is the user for inter-broker communication. + 2. Pass the JAAS config file location as JVM parameter to each Kafka broker: + + -Djava.security.auth.login.config=/etc/kafka/kafka_server_jaas.conf + + 3. Configure SASL port and SASL mechanisms in server.properties as described here. For example: + + listeners=SASL_SSL://host.name:port + security.inter.broker.protocol=SASL_SSL + sasl.mechanism.inter.broker.protocol=SCRAM-SHA-256 (or SCRAM-SHA-512) + sasl.enabled.mechanisms=SCRAM-SHA-256 (or SCRAM-SHA-512) + + 3. ##### Configuring Kafka Clients + +To configure SASL authentication on the clients: + 1. Configure the JAAS configuration property for each client in producer.properties or consumer.properties. The login module describes how the clients like producer and consumer can connect to the Kafka Broker. The following is an example configuration for a client for the SCRAM mechanisms: + + sasl.jaas.config=org.apache.kafka.common.security.scram.ScramLoginModule required \ + username="alice" \ + password="alice-secret"; + +The options `username` and `password` are used by clients to configure the user for client connections. In this example, clients connect to the broker as user _alice_. Different clients within a JVM may connect as different users by specifying different user names and passwords in `sasl.jaas.config`. + +JAAS configuration for clients may alternatively be specified as a JVM parameter similar to brokers as described here. Clients use the login section named `KafkaClient`. This option allows only one user for all client connections from a JVM. + + 2. Configure the following properties in producer.properties or consumer.properties: + + security.protocol=SASL_SSL + sasl.mechanism=SCRAM-SHA-256 (or SCRAM-SHA-512) + + 4. ##### Security Considerations for SASL/SCRAM + + * The default implementation of SASL/SCRAM in Kafka stores SCRAM credentials in Zookeeper. This is suitable for production use in installations where Zookeeper is secure and on a private network. + * Kafka supports only the strong hash functions SHA-256 and SHA-512 with a minimum iteration count of 4096. Strong hash functions combined with strong passwords and high iteration counts protect against brute force attacks if Zookeeper security is compromised. + * SCRAM should be used only with TLS-encryption to prevent interception of SCRAM exchanges. This protects against dictionary or brute force attacks and against impersonation if Zookeeper is compromised. + * From Kafka version 2.0 onwards, the default SASL/SCRAM credential store may be overridden using custom callback handlers by configuring `sasl.server.callback.handler.class` in installations where Zookeeper is not secure. + * For more details on security considerations, refer to [RFC 5802](https://tools.ietf.org/html/rfc5802#section-9). + 6. #### Authentication using SASL/OAUTHBEARER + +The [OAuth 2 Authorization Framework](https://tools.ietf.org/html/rfc6749) "enables a third-party application to obtain limited access to an HTTP service, either on behalf of a resource owner by orchestrating an approval interaction between the resource owner and the HTTP service, or by allowing the third-party application to obtain access on its own behalf." The SASL OAUTHBEARER mechanism enables the use of the framework in a SASL (i.e. a non-HTTP) context; it is defined in [RFC 7628](https://tools.ietf.org/html/rfc7628). The default OAUTHBEARER implementation in Kafka creates and validates [Unsecured JSON Web Tokens](https://tools.ietf.org/html/rfc7515#appendix-A.5) and is only suitable for use in non-production Kafka installations. Refer to Security Considerations for more details. + +Under the default implementation of `principal.builder.class`, the principalName of OAuthBearerToken is used as the authenticated `Principal` for configuration of ACLs etc. + 1. ##### Configuring Kafka Brokers + + 1. Add a suitably modified JAAS file similar to the one below to each Kafka broker's config directory, let's call it kafka_server_jaas.conf for this example: + + KafkaServer { + org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule required + unsecuredLoginStringClaim_sub="admin"; + }; + +The property `unsecuredLoginStringClaim_sub` in the `KafkaServer` section is used by the broker when it initiates connections to other brokers. In this example, _admin_ will appear in the subject (`sub`) claim and will be the user for inter-broker communication. + 2. Pass the JAAS config file location as JVM parameter to each Kafka broker: + + -Djava.security.auth.login.config=/etc/kafka/kafka_server_jaas.conf + + 3. Configure SASL port and SASL mechanisms in server.properties as described here. For example: + + listeners=SASL_SSL://host.name:port (or SASL_PLAINTEXT if non-production) + security.inter.broker.protocol=SASL_SSL (or SASL_PLAINTEXT if non-production) + sasl.mechanism.inter.broker.protocol=OAUTHBEARER + sasl.enabled.mechanisms=OAUTHBEARER + + 2. ##### Configuring Kafka Clients + +To configure SASL authentication on the clients: + 1. Configure the JAAS configuration property for each client in producer.properties or consumer.properties. The login module describes how the clients like producer and consumer can connect to the Kafka Broker. The following is an example configuration for a client for the OAUTHBEARER mechanisms: + + sasl.jaas.config=org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule required \ + unsecuredLoginStringClaim_sub="alice"; + +The option `unsecuredLoginStringClaim_sub` is used by clients to configure the subject (`sub`) claim, which determines the user for client connections. In this example, clients connect to the broker as user _alice_. Different clients within a JVM may connect as different users by specifying different subject (`sub`) claims in `sasl.jaas.config`. + +JAAS configuration for clients may alternatively be specified as a JVM parameter similar to brokers as described here. Clients use the login section named `KafkaClient`. This option allows only one user for all client connections from a JVM. + + 2. Configure the following properties in producer.properties or consumer.properties: + + security.protocol=SASL_SSL (or SASL_PLAINTEXT if non-production) + sasl.mechanism=OAUTHBEARER + + 3. The default implementation of SASL/OAUTHBEARER depends on the jackson-databind library. Since it's an optional dependency, users have to configure it as a dependency via their build tool. + 3. ##### Unsecured Token Creation Options for SASL/OAUTHBEARER + + * The default implementation of SASL/OAUTHBEARER in Kafka creates and validates [Unsecured JSON Web Tokens](https://tools.ietf.org/html/rfc7515#appendix-A.5). While suitable only for non-production use, it does provide the flexibility to create arbitrary tokens in a DEV or TEST environment. + * Here are the various supported JAAS module options on the client side (and on the broker side if OAUTHBEARER is the inter-broker protocol): JAAS Module Option for Unsecured Token Creation | Documentation +---|--- +`unsecuredLoginStringClaim_="value"` | Creates a `String` claim with the given name and value. Any valid claim name can be specified except '`iat`' and '`exp`' (these are automatically generated). +`unsecuredLoginNumberClaim_="value"` | Creates a `Number` claim with the given name and value. Any valid claim name can be specified except '`iat`' and '`exp`' (these are automatically generated). +`unsecuredLoginListClaim_="value"` | Creates a `String List` claim with the given name and values parsed from the given value where the first character is taken as the delimiter. For example: `unsecuredLoginListClaim_fubar="|value1|value2"`. Any valid claim name can be specified except '`iat`' and '`exp`' (these are automatically generated). +`unsecuredLoginExtension_="value"` | Creates a `String` extension with the given name and value. For example: `unsecuredLoginExtension_traceId="123"`. A valid extension name is any sequence of lowercase or uppercase alphabet characters. In addition, the "auth" extension name is reserved. A valid extension value is any combination of characters with ASCII codes 1-127. +`unsecuredLoginPrincipalClaimName` | Set to a custom claim name if you wish the name of the `String` claim holding the principal name to be something other than '`sub`'. +`unsecuredLoginLifetimeSeconds` | Set to an integer value if the token expiration is to be set to something other than the default value of 3600 seconds (which is 1 hour). The '`exp`' claim will be set to reflect the expiration time. +`unsecuredLoginScopeClaimName` | Set to a custom claim name if you wish the name of the `String` or `String List` claim holding any token scope to be something other than '`scope`'. + 4. ##### Unsecured Token Validation Options for SASL/OAUTHBEARER + + * Here are the various supported JAAS module options on the broker side for [Unsecured JSON Web Token](https://tools.ietf.org/html/rfc7515#appendix-A.5) validation: JAAS Module Option for Unsecured Token Validation | Documentation +---|--- +`unsecuredValidatorPrincipalClaimName="value"` | Set to a non-empty value if you wish a particular `String` claim holding a principal name to be checked for existence; the default is to check for the existence of the '`sub`' claim. +`unsecuredValidatorScopeClaimName="value"` | Set to a custom claim name if you wish the name of the `String` or `String List` claim holding any token scope to be something other than '`scope`'. +`unsecuredValidatorRequiredScope="value"` | Set to a space-delimited list of scope values if you wish the `String/String List` claim holding the token scope to be checked to make sure it contains certain values. +`unsecuredValidatorAllowableClockSkewMs="value"` | Set to a positive integer value if you wish to allow up to some number of positive milliseconds of clock skew (the default is 0). + * The default unsecured SASL/OAUTHBEARER implementation may be overridden (and must be overridden in production environments) using custom login and SASL Server callback handlers. + * For more details on security considerations, refer to [RFC 6749, Section 10](https://tools.ietf.org/html/rfc6749#section-10). + 5. ##### Token Refresh for SASL/OAUTHBEARER + +Kafka periodically refreshes any token before it expires so that the client can continue to make connections to brokers. The parameters that impact how the refresh algorithm operates are specified as part of the producer/consumer/broker configuration and are as follows. See the documentation for these properties elsewhere for details. The default values are usually reasonable, in which case these configuration parameters would not need to be explicitly set. Producer/Consumer/Broker Configuration Property +--- +`sasl.login.refresh.window.factor` +`sasl.login.refresh.window.jitter` +`sasl.login.refresh.min.period.seconds` +`sasl.login.refresh.min.buffer.seconds` + 6. ##### Secure/Production Use of SASL/OAUTHBEARER + +Production use cases will require writing an implementation of `org.apache.kafka.common.security.auth.AuthenticateCallbackHandler` that can handle an instance of `org.apache.kafka.common.security.oauthbearer.OAuthBearerTokenCallback` and declaring it via either the `sasl.login.callback.handler.class` configuration option for a non-broker client or via the `listener.name.sasl_ssl.oauthbearer.sasl.login.callback.handler.class` configuration option for brokers (when SASL/OAUTHBEARER is the inter-broker protocol). + +Production use cases will also require writing an implementation of `org.apache.kafka.common.security.auth.AuthenticateCallbackHandler` that can handle an instance of `org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback` and declaring it via the `listener.name.sasl_ssl.oauthbearer.sasl.server.callback.handler.class` broker configuration option. + 7. ##### Security Considerations for SASL/OAUTHBEARER + + * The default implementation of SASL/OAUTHBEARER in Kafka creates and validates [Unsecured JSON Web Tokens](https://tools.ietf.org/html/rfc7515#appendix-A.5). This is suitable only for non-production use. + * OAUTHBEARER should be used in production enviromnments only with TLS-encryption to prevent interception of tokens. + * The default unsecured SASL/OAUTHBEARER implementation may be overridden (and must be overridden in production environments) using custom login and SASL Server callback handlers as described above. + * For more details on OAuth 2 security considerations in general, refer to [RFC 6749, Section 10](https://tools.ietf.org/html/rfc6749#section-10). + 7. #### Enabling multiple SASL mechanisms in a broker + + 1. Specify configuration for the login modules of all enabled mechanisms in the `KafkaServer` section of the JAAS config file. For example: + + KafkaServer { + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + storeKey=true + keyTab="/etc/security/keytabs/kafka_server.keytab" + principal="kafka/kafka1.hostname.com@EXAMPLE.COM"; + + org.apache.kafka.common.security.plain.PlainLoginModule required + username="admin" + password="admin-secret" + user_admin="admin-secret" + user_alice="alice-secret"; + }; + + 2. Enable the SASL mechanisms in server.properties: + + sasl.enabled.mechanisms=GSSAPI,PLAIN,SCRAM-SHA-256,SCRAM-SHA-512,OAUTHBEARER + + 3. Specify the SASL security protocol and mechanism for inter-broker communication in server.properties if required: + + security.inter.broker.protocol=SASL_PLAINTEXT (or SASL_SSL) + sasl.mechanism.inter.broker.protocol=GSSAPI (or one of the other enabled mechanisms) + + 4. Follow the mechanism-specific steps in GSSAPI (Kerberos), PLAIN, SCRAM and OAUTHBEARER to configure SASL for the enabled mechanisms. + 8. #### Modifying SASL mechanism in a Running Cluster + +SASL mechanism can be modified in a running cluster using the following sequence: + + 1. Enable new SASL mechanism by adding the mechanism to `sasl.enabled.mechanisms` in server.properties for each broker. Update JAAS config file to include both mechanisms as described here. Incrementally bounce the cluster nodes. + 2. Restart clients using the new mechanism. + 3. To change the mechanism of inter-broker communication (if this is required), set `sasl.mechanism.inter.broker.protocol` in server.properties to the new mechanism and incrementally bounce the cluster again. + 4. To remove old mechanism (if this is required), remove the old mechanism from `sasl.enabled.mechanisms` in server.properties and remove the entries for the old mechanism from JAAS config file. Incrementally bounce the cluster again. + 9. #### Authentication using Delegation Tokens + +Delegation token based authentication is a lightweight authentication mechanism to complement existing SASL/SSL methods. Delegation tokens are shared secrets between kafka brokers and clients. Delegation tokens will help processing frameworks to distribute the workload to available workers in a secure environment without the added cost of distributing Kerberos TGT/keytabs or keystores when 2-way SSL is used. See [KIP-48](https://cwiki.apache.org/confluence/display/KAFKA/KIP-48+Delegation+token+support+for+Kafka) for more details. + +Under the default implementation of `principal.builder.class`, the owner of delegation token is used as the authenticated `Principal` for configuration of ACLs etc. + +Typical steps for delegation token usage are: + + 1. User authenticates with the Kafka cluster via SASL or SSL, and obtains a delegation token. This can be done using Admin APIs or using `kafka-delegation-tokens.sh` script. + 2. User securely passes the delegation token to Kafka clients for authenticating with the Kafka cluster. + 3. Token owner/renewer can renew/expire the delegation tokens. + 1. ##### Token Management + +A secret is used to generate and verify delegation tokens. This is supplied using config option `delegation.token.secret.key`. The same secret key must be configured across all the brokers. If using Kafka with KRaft the controllers must also be configured with the secret using the same config option. If the secret is not set or set to empty string, delegation token authentication and API operations will fail. + +When using Kafka with Zookeeper, the token details are stored in Zookeeper and delegation tokens are suitable for use in Kafka installations where Zookeeper is on a private network. When using Kafka with KRaft, the token details are stored with the other metadata on the controller nodes and delegation tokens are suitable for use when the controllers are on a private network or when all commnications between brokers and controllers is encrypted. Currently, this secret is stored as plain text in the server.properties config file. We intend to make these configurable in a future Kafka release. + +A token has a current life, and a maximum renewable life. By default, tokens must be renewed once every 24 hours for up to 7 days. These can be configured using `delegation.token.expiry.time.ms` and `delegation.token.max.lifetime.ms` config options. + +Tokens can also be cancelled explicitly. If a token is not renewed by the token’s expiration time or if token is beyond the max life time, it will be deleted from all broker caches as well as from zookeeper. + + 2. ##### Creating Delegation Tokens + +Tokens can be created by using Admin APIs or using `kafka-delegation-tokens.sh` script. Delegation token requests (create/renew/expire/describe) should be issued only on SASL or SSL authenticated channels. Tokens can not be requests if the initial authentication is done through delegation token. A token can be created by the user for that user or others as well by specifying the `--owner-principal` parameter. Owner/Renewers can renew or expire tokens. Owner/renewers can always describe their own tokens. To describe other tokens, a DESCRIBE_TOKEN permission needs to be added on the User resource representing the owner of the token. `kafka-delegation-tokens.sh` script examples are given below. + +Create a delegation token: + + > bin/kafka-delegation-tokens.sh --bootstrap-server localhost:9092 --create --max-life-time-period -1 --command-config client.properties --renewer-principal User:user1 + +Create a delegation token for a different owner: + + > bin/kafka-delegation-tokens.sh --bootstrap-server localhost:9092 --create --max-life-time-period -1 --command-config client.properties --renewer-principal User:user1 --owner-principal User:owner1 + +Renew a delegation token: + + > bin/kafka-delegation-tokens.sh --bootstrap-server localhost:9092 --renew --renew-time-period -1 --command-config client.properties --hmac ABCDEFGHIJK + +Expire a delegation token: + + > bin/kafka-delegation-tokens.sh --bootstrap-server localhost:9092 --expire --expiry-time-period -1 --command-config client.properties --hmac ABCDEFGHIJK + +Existing tokens can be described using the --describe option: + + > bin/kafka-delegation-tokens.sh --bootstrap-server localhost:9092 --describe --command-config client.properties --owner-principal User:user1 + + 3. ##### Token Authentication + +Delegation token authentication piggybacks on the current SASL/SCRAM authentication mechanism. We must enable SASL/SCRAM mechanism on Kafka cluster as described in here. + +Configuring Kafka Clients: + + 1. Configure the JAAS configuration property for each client in producer.properties or consumer.properties. The login module describes how the clients like producer and consumer can connect to the Kafka Broker. The following is an example configuration for a client for the token authentication: + + sasl.jaas.config=org.apache.kafka.common.security.scram.ScramLoginModule required \ + username="tokenID123" \ + password="lAYYSFmLs4bTjf+lTZ1LCHR/ZZFNA==" \ + tokenauth="true"; + +The options `username` and `password` are used by clients to configure the token id and token HMAC. And the option `tokenauth` is used to indicate the server about token authentication. In this example, clients connect to the broker using token id: _tokenID123_. Different clients within a JVM may connect using different tokens by specifying different token details in `sasl.jaas.config`. + +JAAS configuration for clients may alternatively be specified as a JVM parameter similar to brokers as described here. Clients use the login section named `KafkaClient`. This option allows only one user for all client connections from a JVM. + + 4. ##### Procedure to manually rotate the secret: + +We require a re-deployment when the secret needs to be rotated. During this process, already connected clients will continue to work. But any new connection requests and renew/expire requests with old tokens can fail. Steps are given below. + + 1. Expire all existing tokens. + 2. Rotate the secret by rolling upgrade, and + 3. Generate new tokens + +We intend to automate this in a future Kafka release. + + + diff --git a/docs/security/authorization-and-acls.md b/docs/security/authorization-and-acls.md new file mode 100644 index 0000000000000..73b237cb6a7ac --- /dev/null +++ b/docs/security/authorization-and-acls.md @@ -0,0 +1,330 @@ +--- +title: Authorization and ACLs +description: Authorization and ACLs +weight: 5 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +Kafka ships with a pluggable authorization framework, which is configured with the `authorizer.class.name` property in the server configuration. Configured implementations must extend `org.apache.kafka.server.authorizer.Authorizer`. Kafka provides default implementations which store ACLs in the cluster metadata (either Zookeeper or the KRaft metadata log). For Zookeeper-based clusters, the provided implementation is configured as follows: + + + authorizer.class.name=kafka.security.authorizer.AclAuthorizer + +For KRaft clusters, use the following configuration on all nodes (brokers, controllers, or combined broker/controller nodes): + + + authorizer.class.name=org.apache.kafka.metadata.authorizer.StandardAuthorizer + +Kafka ACLs are defined in the general format of "Principal {P} is [Allowed|Denied] Operation {O} From Host {H} on any Resource {R} matching ResourcePattern {RP}". You can read more about the ACL structure in [KIP-11](https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface) and resource patterns in [KIP-290](https://cwiki.apache.org/confluence/display/KAFKA/KIP-290%3A+Support+for+Prefixed+ACLs). In order to add, remove, or list ACLs, you can use the Kafka ACL CLI `kafka-acls.sh`. By default, if no ResourcePatterns match a specific Resource R, then R has no associated ACLs, and therefore no one other than super users is allowed to access R. If you want to change that behavior, you can include the following in server.properties. + + + allow.everyone.if.no.acl.found=true + +One can also add super users in server.properties like the following (note that the delimiter is semicolon since SSL user names may contain comma). Default PrincipalType string "User" is case sensitive. + + + super.users=User:Bob;User:Alice + +### KRaft Principal Forwarding + +In KRaft clusters, admin requests such as `CreateTopics` and `DeleteTopics` are sent to the broker listeners by the client. The broker then forwards the request to the active controller through the first listener configured in `controller.listener.names`. Authorization of these requests is done on the controller node. This is achieved by way of an `Envelope` request which packages both the underlying request from the client as well as the client principal. When the controller receives the forwarded `Envelope` request from the broker, it first authorizes the `Envelope` request using the authenticated broker principal. Then it authorizes the underlying request using the forwarded principal. +All of this implies that Kafka must understand how to serialize and deserialize the client principal. The authentication framework allows for customized principals by overriding the `principal.builder.class` configuration. In order for customized principals to work with KRaft, the configured class must implement `org.apache.kafka.common.security.auth.KafkaPrincipalSerde` so that Kafka knows how to serialize and deserialize the principals. The default implementation `org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder` uses the Kafka RPC format defined in the source code: `clients/src/main/resources/common/message/DefaultPrincipalData.json`. For more detail about request forwarding in KRaft, see [KIP-590](https://cwiki.apache.org/confluence/display/KAFKA/KIP-590%3A+Redirect+Zookeeper+Mutation+Protocols+to+The+Controller) + +### Customizing SSL User Name + +By default, the SSL user name will be of the form "CN=writeuser,OU=Unknown,O=Unknown,L=Unknown,ST=Unknown,C=Unknown". One can change that by setting `ssl.principal.mapping.rules` to a customized rule in server.properties. This config allows a list of rules for mapping X.500 distinguished name to short name. The rules are evaluated in order and the first rule that matches a distinguished name is used to map it to a short name. Any later rules in the list are ignored. +The format of `ssl.principal.mapping.rules` is a list where each rule starts with "RULE:" and contains an expression as the following formats. Default rule will return string representation of the X.500 certificate distinguished name. If the distinguished name matches the pattern, then the replacement command will be run over the name. This also supports lowercase/uppercase options, to force the translated result to be all lower/uppercase case. This is done by adding a "/L" or "/U' to the end of the rule. + + + RULE:pattern/replacement/ + RULE:pattern/replacement/[LU] + +Example `ssl.principal.mapping.rules` values are: + + + RULE:^CN=(.*?),OU=ServiceUsers.*$/$1/, + RULE:^CN=(.*?),OU=(.*?),O=(.*?),L=(.*?),ST=(.*?),C=(.*?)$/$1@$2/L, + RULE:^.*[Cc][Nn]=([a-zA-Z0-9.]*).*$/$1/L, + DEFAULT + +Above rules translate distinguished name "CN=serviceuser,OU=ServiceUsers,O=Unknown,L=Unknown,ST=Unknown,C=Unknown" to "serviceuser" and "CN=adminUser,OU=Admin,O=Unknown,L=Unknown,ST=Unknown,C=Unknown" to "adminuser@admin". +For advanced use cases, one can customize the name by setting a customized PrincipalBuilder in server.properties like the following. + + + principal.builder.class=CustomizedPrincipalBuilderClass + +### Customizing SASL User Name + +By default, the SASL user name will be the primary part of the Kerberos principal. One can change that by setting `sasl.kerberos.principal.to.local.rules` to a customized rule in server.properties. The format of `sasl.kerberos.principal.to.local.rules` is a list where each rule works in the same way as the auth_to_local in [Kerberos configuration file (krb5.conf)](https://web.mit.edu/Kerberos/krb5-latest/doc/admin/conf_files/krb5_conf.html). This also support additional lowercase/uppercase rule, to force the translated result to be all lowercase/uppercase. This is done by adding a "/L" or "/U" to the end of the rule. check below formats for syntax. Each rules starts with RULE: and contains an expression as the following formats. See the kerberos documentation for more details. + + + RULE:[n:string](regexp)s/pattern/replacement/ + RULE:[n:string](regexp)s/pattern/replacement/g + RULE:[n:string](regexp)s/pattern/replacement//L + RULE:[n:string](regexp)s/pattern/replacement/g/L + RULE:[n:string](regexp)s/pattern/replacement//U + RULE:[n:string](regexp)s/pattern/replacement/g/U + +An example of adding a rule to properly translate user@MYDOMAIN.COM to user while also keeping the default rule in place is: + + + sasl.kerberos.principal.to.local.rules=RULE:[1:$1@$0](.*@MYDOMAIN.COM)s/@.*//,DEFAULT + +## Command Line Interface + +Kafka Authorization management CLI can be found under bin directory with all the other CLIs. The CLI script is called **kafka-acls.sh**. Following lists all the options that the script supports: + +Option | Description | Default | Option type +---|---|---|--- +\--add | Indicates to the script that user is trying to add an acl. | | Action +\--remove | Indicates to the script that user is trying to remove an acl. | | Action +\--list | Indicates to the script that user is trying to list acls. | | Action +\--bootstrap-server | A list of host/port pairs to use for establishing the connection to the Kafka cluster. Only one of --bootstrap-server or --authorizer option must be specified. | | Configuration +\--command-config | A property file containing configs to be passed to Admin Client. This option can only be used with --bootstrap-server option. | | Configuration +\--cluster | Indicates to the script that the user is trying to interact with acls on the singular cluster resource. | | ResourcePattern +\--topic [topic-name] | Indicates to the script that the user is trying to interact with acls on topic resource pattern(s). | | ResourcePattern +\--group [group-name] | Indicates to the script that the user is trying to interact with acls on consumer-group resource pattern(s) | | ResourcePattern +\--transactional-id [transactional-id] | The transactionalId to which ACLs should be added or removed. A value of * indicates the ACLs should apply to all transactionalIds. | | ResourcePattern +\--delegation-token [delegation-token] | Delegation token to which ACLs should be added or removed. A value of * indicates ACL should apply to all tokens. | | ResourcePattern +\--user-principal [user-principal] | A user resource to which ACLs should be added or removed. This is currently supported in relation with delegation tokens. A value of * indicates ACL should apply to all users. | | ResourcePattern +\--resource-pattern-type [pattern-type] | Indicates to the script the type of resource pattern, (for --add), or resource pattern filter, (for --list and --remove), the user wishes to use. +When adding acls, this should be a specific pattern type, e.g. 'literal' or 'prefixed'. +When listing or removing acls, a specific pattern type filter can be used to list or remove acls from a specific type of resource pattern, or the filter values of 'any' or 'match' can be used, where 'any' will match any pattern type, but will match the resource name exactly, and 'match' will perform pattern matching to list or remove all acls that affect the supplied resource(s). +WARNING: 'match', when used in combination with the '--remove' switch, should be used with care. | literal | Configuration +\--allow-principal | Principal is in PrincipalType:name format that will be added to ACL with Allow permission. Default PrincipalType string "User" is case sensitive. +You can specify multiple --allow-principal in a single command. | | Principal +\--deny-principal | Principal is in PrincipalType:name format that will be added to ACL with Deny permission. Default PrincipalType string "User" is case sensitive. +You can specify multiple --deny-principal in a single command. | | Principal +\--principal | Principal is in PrincipalType:name format that will be used along with --list option. Default PrincipalType string "User" is case sensitive. This will list the ACLs for the specified principal. +You can specify multiple --principal in a single command. | | Principal +\--allow-host | IP address from which principals listed in --allow-principal will have access. | if --allow-principal is specified defaults to * which translates to "all hosts" | Host +\--deny-host | IP address from which principals listed in --deny-principal will be denied access. | if --deny-principal is specified defaults to * which translates to "all hosts" | Host +\--operation | Operation that will be allowed or denied. +Valid values are: + + * Read + * Write + * Create + * Delete + * Alter + * Describe + * ClusterAction + * DescribeConfigs + * AlterConfigs + * IdempotentWrite + * CreateTokens + * DescribeTokens + * All + +| All | Operation +\--producer | Convenience option to add/remove acls for producer role. This will generate acls that allows WRITE, DESCRIBE and CREATE on topic. | | Convenience +\--consumer | Convenience option to add/remove acls for consumer role. This will generate acls that allows READ, DESCRIBE on topic and READ on consumer-group. | | Convenience +\--idempotent | Enable idempotence for the producer. This should be used in combination with the --producer option. +Note that idempotence is enabled automatically if the producer is authorized to a particular transactional-id. | | Convenience +\--force | Convenience option to assume yes to all queries and do not prompt. | | Convenience +\--authorizer | (DEPRECATED: not supported in KRaft) Fully qualified class name of the authorizer. | kafka.security.authorizer.AclAuthorizer | Configuration +\--authorizer-properties | (DEPRECATED: not supported in KRaft) key=val pairs that will be passed to authorizer for initialization. For the default authorizer in ZK clsuters, the example values are: zookeeper.connect=localhost:2181 | | Configuration +\--zk-tls-config-file | (DEPRECATED: not supported in KRaft) Identifies the file where ZooKeeper client TLS connectivity properties for the authorizer are defined. Any properties other than the following (with or without an "authorizer." prefix) are ignored: zookeeper.clientCnxnSocket, zookeeper.ssl.cipher.suites, zookeeper.ssl.client.enable, zookeeper.ssl.crl.enable, zookeeper.ssl.enabled.protocols, zookeeper.ssl.endpoint.identification.algorithm, zookeeper.ssl.keystore.location, zookeeper.ssl.keystore.password, zookeeper.ssl.keystore.type, zookeeper.ssl.ocsp.enable, zookeeper.ssl.protocol, zookeeper.ssl.truststore.location, zookeeper.ssl.truststore.password, zookeeper.ssl.truststore.type | | Configuration + +## Examples + + * **Adding Acls** +Suppose you want to add an acl "Principals User:Bob and User:Alice are allowed to perform Operation Read and Write on Topic Test-Topic from IP 198.51.100.0 and IP 198.51.100.1". You can do that by executing the CLI with following options: + + > bin/kafka-acls.sh --bootstrap-server localhost:9092 --add --allow-principal User:Bob --allow-principal User:Alice --allow-host 198.51.100.0 --allow-host 198.51.100.1 --operation Read --operation Write --topic Test-topic + +By default, all principals that don't have an explicit acl that allows access for an operation to a resource are denied. In rare cases where an allow acl is defined that allows access to all but some principal we will have to use the --deny-principal and --deny-host option. For example, if we want to allow all users to Read from Test-topic but only deny User:BadBob from IP 198.51.100.3 we can do so using following commands: + + > bin/kafka-acls.sh --bootstrap-server localhost:9092 --add --allow-principal User:'*' --allow-host '*' --deny-principal User:BadBob --deny-host 198.51.100.3 --operation Read --topic Test-topic + +Note that `--allow-host` and `--deny-host` only support IP addresses (hostnames are not supported). Above examples add acls to a topic by specifying --topic [topic-name] as the resource pattern option. Similarly user can add acls to cluster by specifying --cluster and to a consumer group by specifying --group [group-name]. You can add acls on any resource of a certain type, e.g. suppose you wanted to add an acl "Principal User:Peter is allowed to produce to any Topic from IP 198.51.200.0" You can do that by using the wildcard resource '*', e.g. by executing the CLI with following options: + + > bin/kafka-acls.sh --bootstrap-server localhost:9092 --add --allow-principal User:Peter --allow-host 198.51.200.1 --producer --topic '*' + +You can add acls on prefixed resource patterns, e.g. suppose you want to add an acl "Principal User:Jane is allowed to produce to any Topic whose name starts with 'Test-' from any host". You can do that by executing the CLI with following options: + + > bin/kafka-acls.sh --bootstrap-server localhost:9092 --add --allow-principal User:Jane --producer --topic Test- --resource-pattern-type prefixed + +Note, --resource-pattern-type defaults to 'literal', which only affects resources with the exact same name or, in the case of the wildcard resource name '*', a resource with any name. + * **Removing Acls** +Removing acls is pretty much the same. The only difference is instead of --add option users will have to specify --remove option. To remove the acls added by the first example above we can execute the CLI with following options: + + > bin/kafka-acls.sh --bootstrap-server localhost:9092 --remove --allow-principal User:Bob --allow-principal User:Alice --allow-host 198.51.100.0 --allow-host 198.51.100.1 --operation Read --operation Write --topic Test-topic + +If you want to remove the acl added to the prefixed resource pattern above we can execute the CLI with following options: + + > bin/kafka-acls.sh --bootstrap-server localhost:9092 --remove --allow-principal User:Jane --producer --topic Test- --resource-pattern-type Prefixed + + * **List Acls** +We can list acls for any resource by specifying the --list option with the resource. To list all acls on the literal resource pattern Test-topic, we can execute the CLI with following options: + + > bin/kafka-acls.sh --bootstrap-server localhost:9092 --list --topic Test-topic + +However, this will only return the acls that have been added to this exact resource pattern. Other acls can exist that affect access to the topic, e.g. any acls on the topic wildcard '*', or any acls on prefixed resource patterns. Acls on the wildcard resource pattern can be queried explicitly: + + > bin/kafka-acls.sh --bootstrap-server localhost:9092 --list --topic '*' + +However, it is not necessarily possible to explicitly query for acls on prefixed resource patterns that match Test-topic as the name of such patterns may not be known. We can list _all_ acls affecting Test-topic by using '--resource-pattern-type match', e.g. + + > bin/kafka-acls.sh --bootstrap-server localhost:9092 --list --topic Test-topic --resource-pattern-type match + +This will list acls on all matching literal, wildcard and prefixed resource patterns. + * **Adding or removing a principal as producer or consumer** +The most common use case for acl management are adding/removing a principal as producer or consumer so we added convenience options to handle these cases. In order to add User:Bob as a producer of Test-topic we can execute the following command: + + > bin/kafka-acls.sh --bootstrap-server localhost:9092 --add --allow-principal User:Bob --producer --topic Test-topic + +Similarly to add Alice as a consumer of Test-topic with consumer group Group-1 we just have to pass --consumer option: + + > bin/kafka-acls.sh --bootstrap-server localhost:9092 --add --allow-principal User:Bob --consumer --topic Test-topic --group Group-1 + +Note that for consumer option we must also specify the consumer group. In order to remove a principal from producer or consumer role we just need to pass --remove option. + * **Admin API based acl management** +Users having Alter permission on ClusterResource can use Admin API for ACL management. kafka-acls.sh script supports AdminClient API to manage ACLs without interacting with zookeeper/authorizer directly. All the above examples can be executed by using **\--bootstrap-server** option. For example: + + bin/kafka-acls.sh --bootstrap-server localhost:9092 --command-config /tmp/adminclient-configs.conf --add --allow-principal User:Bob --producer --topic Test-topic + bin/kafka-acls.sh --bootstrap-server localhost:9092 --command-config /tmp/adminclient-configs.conf --add --allow-principal User:Bob --consumer --topic Test-topic --group Group-1 + bin/kafka-acls.sh --bootstrap-server localhost:9092 --command-config /tmp/adminclient-configs.conf --list --topic Test-topic + bin/kafka-acls.sh --bootstrap-server localhost:9092 --command-config /tmp/adminclient-configs.conf --add --allow-principal User:tokenRequester --operation CreateTokens --user-principal "owner1" + + + + +## Authorization Primitives + +Protocol calls are usually performing some operations on certain resources in Kafka. It is required to know the operations and resources to set up effective protection. In this section we'll list these operations and resources, then list the combination of these with the protocols to see the valid scenarios. + +### Operations in Kafka + +There are a few operation primitives that can be used to build up privileges. These can be matched up with certain resources to allow specific protocol calls for a given user. These are: + + * Read + * Write + * Create + * Delete + * Alter + * Describe + * ClusterAction + * DescribeConfigs + * AlterConfigs + * IdempotentWrite + * CreateTokens + * DescribeTokens + * All + + + +### Resources in Kafka + +The operations above can be applied on certain resources which are described below. + + * **Topic:** this simply represents a Topic. All protocol calls that are acting on topics (such as reading, writing them) require the corresponding privilege to be added. If there is an authorization error with a topic resource, then a TOPIC_AUTHORIZATION_FAILED (error code: 29) will be returned. + * **Group:** this represents the consumer groups in the brokers. All protocol calls that are working with consumer groups, like joining a group must have privileges with the group in subject. If the privilege is not given then a GROUP_AUTHORIZATION_FAILED (error code: 30) will be returned in the protocol response. + * **Cluster:** this resource represents the cluster. Operations that are affecting the whole cluster, like controlled shutdown are protected by privileges on the Cluster resource. If there is an authorization problem on a cluster resource, then a CLUSTER_AUTHORIZATION_FAILED (error code: 31) will be returned. + * **TransactionalId:** this resource represents actions related to transactions, such as committing. If any error occurs, then a TRANSACTIONAL_ID_AUTHORIZATION_FAILED (error code: 53) will be returned by brokers. + * **DelegationToken:** this represents the delegation tokens in the cluster. Actions, such as describing delegation tokens could be protected by a privilege on the DelegationToken resource. Since these objects have a little special behavior in Kafka it is recommended to read [KIP-48](https://cwiki.apache.org/confluence/display/KAFKA/KIP-48+Delegation+token+support+for+Kafka#KIP-48DelegationtokensupportforKafka-DescribeDelegationTokenRequest) and the related upstream documentation at Authentication using Delegation Tokens. + * **User:** CreateToken and DescribeToken operations can be granted to User resources to allow creating and describing tokens for other users. More info can be found in [KIP-373](https://cwiki.apache.org/confluence/display/KAFKA/KIP-373%3A+Allow+users+to+create+delegation+tokens+for+other+users). + + + +### Operations and Resources on Protocols + +In the below table we'll list the valid operations on resources that are executed by the Kafka API protocols. + +Protocol (API key) | Operation | Resource | Note +---|---|---|--- +PRODUCE (0) | Write | TransactionalId | An transactional producer which has its transactional.id set requires this privilege. +PRODUCE (0) | IdempotentWrite | Cluster | An idempotent produce action requires this privilege. +PRODUCE (0) | Write | Topic | This applies to a normal produce action. +FETCH (1) | ClusterAction | Cluster | A follower must have ClusterAction on the Cluster resource in order to fetch partition data. +FETCH (1) | Read | Topic | Regular Kafka consumers need READ permission on each partition they are fetching. +LIST_OFFSETS (2) | Describe | Topic | +METADATA (3) | Describe | Topic | +METADATA (3) | Create | Cluster | If topic auto-creation is enabled, then the broker-side API will check for the existence of a Cluster level privilege. If it's found then it'll allow creating the topic, otherwise it'll iterate through the Topic level privileges (see the next one). +METADATA (3) | Create | Topic | This authorizes auto topic creation if enabled but the given user doesn't have a cluster level permission (above). +LEADER_AND_ISR (4) | ClusterAction | Cluster | +STOP_REPLICA (5) | ClusterAction | Cluster | +UPDATE_METADATA (6) | ClusterAction | Cluster | +CONTROLLED_SHUTDOWN (7) | ClusterAction | Cluster | +OFFSET_COMMIT (8) | Read | Group | An offset can only be committed if it's authorized to the given group and the topic too (see below). Group access is checked first, then Topic access. +OFFSET_COMMIT (8) | Read | Topic | Since offset commit is part of the consuming process, it needs privileges for the read action. +OFFSET_FETCH (9) | Describe | Group | Similarly to OFFSET_COMMIT, the application must have privileges on group and topic level too to be able to fetch. However in this case it requires describe access instead of read. Group access is checked first, then Topic access. +OFFSET_FETCH (9) | Describe | Topic | +FIND_COORDINATOR (10) | Describe | Group | The FIND_COORDINATOR request can be of "Group" type in which case it is looking for consumergroup coordinators. This privilege would represent the Group mode. +FIND_COORDINATOR (10) | Describe | TransactionalId | This applies only on transactional producers and checked when a producer tries to find the transaction coordinator. +JOIN_GROUP (11) | Read | Group | +HEARTBEAT (12) | Read | Group | +LEAVE_GROUP (13) | Read | Group | +SYNC_GROUP (14) | Read | Group | +DESCRIBE_GROUPS (15) | Describe | Group | +LIST_GROUPS (16) | Describe | Cluster | When the broker checks to authorize a list_groups request it first checks for this cluster level authorization. If none found then it proceeds to check the groups individually. This operation doesn't return CLUSTER_AUTHORIZATION_FAILED. +LIST_GROUPS (16) | Describe | Group | If none of the groups are authorized, then just an empty response will be sent back instead of an error. This operation doesn't return CLUSTER_AUTHORIZATION_FAILED. This is applicable from the 2.1 release. +SASL_HANDSHAKE (17) | | | The SASL handshake is part of the authentication process and therefore it's not possible to apply any kind of authorization here. +API_VERSIONS (18) | | | The API_VERSIONS request is part of the Kafka protocol handshake and happens on connection and before any authentication. Therefore it's not possible to control this with authorization. +CREATE_TOPICS (19) | Create | Cluster | If there is no cluster level authorization then it won't return CLUSTER_AUTHORIZATION_FAILED but fall back to use topic level, which is just below. That'll throw error if there is a problem. +CREATE_TOPICS (19) | Create | Topic | This is applicable from the 2.0 release. +DELETE_TOPICS (20) | Delete | Topic | +DELETE_RECORDS (21) | Delete | Topic | +INIT_PRODUCER_ID (22) | Write | TransactionalId | +INIT_PRODUCER_ID (22) | IdempotentWrite | Cluster | +OFFSET_FOR_LEADER_EPOCH (23) | ClusterAction | Cluster | If there is no cluster level privilege for this operation, then it'll check for topic level one. +OFFSET_FOR_LEADER_EPOCH (23) | Describe | Topic | This is applicable from the 2.1 release. +ADD_PARTITIONS_TO_TXN (24) | Write | TransactionalId | This API is only applicable to transactional requests. It first checks for the Write action on the TransactionalId resource, then it checks the Topic in subject (below). +ADD_PARTITIONS_TO_TXN (24) | Write | Topic | +ADD_OFFSETS_TO_TXN (25) | Write | TransactionalId | Similarly to ADD_PARTITIONS_TO_TXN this is only applicable to transactional request. It first checks for Write action on the TransactionalId resource, then it checks whether it can Read on the given group (below). +ADD_OFFSETS_TO_TXN (25) | Read | Group | +END_TXN (26) | Write | TransactionalId | +WRITE_TXN_MARKERS (27) | ClusterAction | Cluster | +TXN_OFFSET_COMMIT (28) | Write | TransactionalId | +TXN_OFFSET_COMMIT (28) | Read | Group | +TXN_OFFSET_COMMIT (28) | Read | Topic | +DESCRIBE_ACLS (29) | Describe | Cluster | +CREATE_ACLS (30) | Alter | Cluster | +DELETE_ACLS (31) | Alter | Cluster | +DESCRIBE_CONFIGS (32) | DescribeConfigs | Cluster | If broker configs are requested, then the broker will check cluster level privileges. +DESCRIBE_CONFIGS (32) | DescribeConfigs | Topic | If topic configs are requested, then the broker will check topic level privileges. +ALTER_CONFIGS (33) | AlterConfigs | Cluster | If broker configs are altered, then the broker will check cluster level privileges. +ALTER_CONFIGS (33) | AlterConfigs | Topic | If topic configs are altered, then the broker will check topic level privileges. +ALTER_REPLICA_LOG_DIRS (34) | Alter | Cluster | +DESCRIBE_LOG_DIRS (35) | Describe | Cluster | An empty response will be returned on authorization failure. +SASL_AUTHENTICATE (36) | | | SASL_AUTHENTICATE is part of the authentication process and therefore it's not possible to apply any kind of authorization here. +CREATE_PARTITIONS (37) | Alter | Topic | +CREATE_DELEGATION_TOKEN (38) | | | Creating delegation tokens has special rules, for this please see the Authentication using Delegation Tokens section. +CREATE_DELEGATION_TOKEN (38) | CreateTokens | User | Allows creating delegation tokens for the User resource. +RENEW_DELEGATION_TOKEN (39) | | | Renewing delegation tokens has special rules, for this please see the Authentication using Delegation Tokens section. +EXPIRE_DELEGATION_TOKEN (40) | | | Expiring delegation tokens has special rules, for this please see the Authentication using Delegation Tokens section. +DESCRIBE_DELEGATION_TOKEN (41) | Describe | DelegationToken | Describing delegation tokens has special rules, for this please see the Authentication using Delegation Tokens section. +DESCRIBE_DELEGATION_TOKEN (41) | DescribeTokens | User | Allows describing delegation tokens of the User resource. +DELETE_GROUPS (42) | Delete | Group | +ELECT_PREFERRED_LEADERS (43) | ClusterAction | Cluster | +INCREMENTAL_ALTER_CONFIGS (44) | AlterConfigs | Cluster | If broker configs are altered, then the broker will check cluster level privileges. +INCREMENTAL_ALTER_CONFIGS (44) | AlterConfigs | Topic | If topic configs are altered, then the broker will check topic level privileges. +ALTER_PARTITION_REASSIGNMENTS (45) | Alter | Cluster | +LIST_PARTITION_REASSIGNMENTS (46) | Describe | Cluster | +OFFSET_DELETE (47) | Delete | Group | +OFFSET_DELETE (47) | Read | Topic | +DESCRIBE_CLIENT_QUOTAS (48) | DescribeConfigs | Cluster | +ALTER_CLIENT_QUOTAS (49) | AlterConfigs | Cluster | +DESCRIBE_USER_SCRAM_CREDENTIALS (50) | Describe | Cluster | +ALTER_USER_SCRAM_CREDENTIALS (51) | Alter | Cluster | +VOTE (52) | ClusterAction | Cluster | +BEGIN_QUORUM_EPOCH (53) | ClusterAction | Cluster | +END_QUORUM_EPOCH (54) | ClusterAction | Cluster | +DESCRIBE_QUORUM (55) | Describe | Cluster | +ALTER_PARTITION (56) | ClusterAction | Cluster | +UPDATE_FEATURES (57) | Alter | Cluster | +ENVELOPE (58) | ClusterAction | Cluster | +FETCH_SNAPSHOT (59) | ClusterAction | Cluster | +DESCRIBE_CLUSTER (60) | Describe | Cluster | +DESCRIBE_PRODUCERS (61) | Read | Topic | +BROKER_REGISTRATION (62) | ClusterAction | Cluster | +BROKER_HEARTBEAT (63) | ClusterAction | Cluster | +UNREGISTER_BROKER (64) | Alter | Cluster | +DESCRIBE_TRANSACTIONS (65) | Describe | TransactionalId | +LIST_TRANSACTIONS (66) | Describe | TransactionalId | +ALLOCATE_PRODUCER_IDS (67) | ClusterAction | Cluster | +CONSUMER_GROUP_HEARTBEAT (68) | Read | Group | + \ No newline at end of file diff --git a/docs/security/encryption-and-authentication-using-ssl.md b/docs/security/encryption-and-authentication-using-ssl.md new file mode 100644 index 0000000000000..ead912e1b34e4 --- /dev/null +++ b/docs/security/encryption-and-authentication-using-ssl.md @@ -0,0 +1,286 @@ +--- +title: Encryption and Authentication using SSL +description: Encryption and Authentication using SSL +weight: 3 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +Apache Kafka allows clients to use SSL for encryption of traffic as well as authentication. By default, SSL is disabled but can be turned on if needed. The following paragraphs explain in detail how to set up your own PKI infrastructure, use it to create certificates and configure Kafka to use these. + + 1. #### Generate SSL key and certificate for each Kafka broker + +The first step of deploying one or more brokers with SSL support is to generate a public/private keypair for every server. Since Kafka expects all keys and certificates to be stored in keystores we will use Java's keytool command for this task. The tool supports two different keystore formats, the Java specific jks format which has been deprecated by now, as well as PKCS12. PKCS12 is the default format as of Java version 9, to ensure this format is being used regardless of the Java version in use all following commands explicitly specify the PKCS12 format. + + > keytool -keystore {keystorefile} -alias localhost -validity {validity} -genkey -keyalg RSA -storetype pkcs12 + +You need to specify two parameters in the above command: + 1. keystorefile: the keystore file that stores the keys (and later the certificate) for this broker. The keystore file contains the private and public keys of this broker, therefore it needs to be kept safe. Ideally this step is run on the Kafka broker that the key will be used on, as this key should never be transmitted/leave the server that it is intended for. + 2. validity: the valid time of the key in days. Please note that this differs from the validity period for the certificate, which will be determined in Signing the certificate. You can use the same key to request multiple certificates: if your key has a validity of 10 years, but your CA will only sign certificates that are valid for one year, you can use the same key with 10 certificates over time. + +To obtain a certificate that can be used with the private key that was just created a certificate signing request needs to be created. This signing request, when signed by a trusted CA results in the actual certificate which can then be installed in the keystore and used for authentication purposes. +To generate certificate signing requests run the following command for all server keystores created so far. + + > keytool -keystore server.keystore.jks -alias localhost -validity {validity} -genkey -keyalg RSA -destkeystoretype pkcs12 -ext SAN=DNS:{FQDN},IP:{IPADDRESS1} + +This command assumes that you want to add hostname information to the certificate, if this is not the case, you can omit the extension parameter `-ext SAN=DNS:{FQDN},IP:{IPADDRESS1}`. Please see below for more information on this. + +### Host Name Verification + +Host name verification, when enabled, is the process of checking attributes from the certificate that is presented by the server you are connecting to against the actual hostname or ip address of that server to ensure that you are indeed connecting to the correct server. +The main reason for this check is to prevent man-in-the-middle attacks. For Kafka, this check has been disabled by default for a long time, but as of Kafka 2.0.0 host name verification of servers is enabled by default for client connections as well as inter-broker connections. +Server host name verification may be disabled by setting `ssl.endpoint.identification.algorithm` to an empty string. +For dynamically configured broker listeners, hostname verification may be disabled using `kafka-configs.sh`: + + + > bin/kafka-configs.sh --bootstrap-server localhost:9093 --entity-type brokers --entity-name 0 --alter --add-config "listener.name.internal.ssl.endpoint.identification.algorithm=" + +**Note:** + +Normally there is no good reason to disable hostname verification apart from being the quickest way to "just get it to work" followed by the promise to "fix it later when there is more time"! +Getting hostname verification right is not that hard when done at the right time, but gets much harder once the cluster is up and running - do yourself a favor and do it now! + +If host name verification is enabled, clients will verify the server's fully qualified domain name (FQDN) or ip address against one of the following two fields: + 1. Common Name (CN) + 2. [Subject Alternative Name (SAN)](https://tools.ietf.org/html/rfc5280#section-4.2.1.6) + +While Kafka checks both fields, usage of the common name field for hostname verification has been [deprecated](https://tools.ietf.org/html/rfc2818#section-3.1) since 2000 and should be avoided if possible. In addition the SAN field is much more flexible, allowing for multiple DNS and IP entries to be declared in a certificate. +Another advantage is that if the SAN field is used for hostname verification the common name can be set to a more meaningful value for authorization purposes. Since we need the SAN field to be contained in the signed certificate, it will be specified when generating the signing request. It can also be specified when generating the keypair, but this will not automatically be copied into the signing request. +To add a SAN field append the following argument ` -ext SAN=DNS:{FQDN},IP:{IPADDRESS}` to the keytool command: + + > keytool -keystore server.keystore.jks -alias localhost -validity {validity} -genkey -keyalg RSA -destkeystoretype pkcs12 -ext SAN=DNS:{FQDN},IP:{IPADDRESS1} + + 2. #### Creating your own CA + +After this step each machine in the cluster has a public/private key pair which can already be used to encrypt traffic and a certificate signing request, which is the basis for creating a certificate. To add authentication capabilities this signing request needs to be signed by a trusted authority, which will be created in this step. + +A certificate authority (CA) is responsible for signing certificates. CAs works likes a government that issues passports - the government stamps (signs) each passport so that the passport becomes difficult to forge. Other governments verify the stamps to ensure the passport is authentic. Similarly, the CA signs the certificates, and the cryptography guarantees that a signed certificate is computationally difficult to forge. Thus, as long as the CA is a genuine and trusted authority, the clients have a strong assurance that they are connecting to the authentic machines. + +For this guide we will be our own Certificate Authority. When setting up a production cluster in a corporate environment these certificates would usually be signed by a corporate CA that is trusted throughout the company. Please see Common Pitfalls in Production for some things to consider for this case. + +Due to a [bug](https://www.openssl.org/docs/man1.1.1/man1/x509.html#BUGS) in OpenSSL, the x509 module will not copy requested extension fields from CSRs into the final certificate. Since we want the SAN extension to be present in our certificate to enable hostname verification, we'll use the _ca_ module instead. This requires some additional configuration to be in place before we generate our CA keypair. +Save the following listing into a file called openssl-ca.cnf and adjust the values for validity and common attributes as necessary. + + HOME = . + RANDFILE = $ENV::HOME/.rnd + + #################################################################### + [ ca ] + default_ca = CA_default # The default ca section + + [ CA_default ] + + base_dir = . + certificate = $base_dir/cacert.pem # The CA certificate + private_key = $base_dir/cakey.pem # The CA private key + new_certs_dir = $base_dir # Location for new certs after signing + database = $base_dir/index.txt # Database index file + serial = $base_dir/serial.txt # The current serial number + + default_days = 1000 # How long to certify for + default_crl_days = 30 # How long before next CRL + default_md = sha256 # Use public key default MD + preserve = no # Keep passed DN ordering + + x509_extensions = ca_extensions # The extensions to add to the cert + + email_in_dn = no # Don't concat the email in the DN + copy_extensions = copy # Required to copy SANs from CSR to cert + + #################################################################### + [ req ] + default_bits = 4096 + default_keyfile = cakey.pem + distinguished_name = ca_distinguished_name + x509_extensions = ca_extensions + string_mask = utf8only + + #################################################################### + [ ca_distinguished_name ] + countryName = Country Name (2 letter code) + countryName_default = DE + + stateOrProvinceName = State or Province Name (full name) + stateOrProvinceName_default = Test Province + + localityName = Locality Name (eg, city) + localityName_default = Test Town + + organizationName = Organization Name (eg, company) + organizationName_default = Test Company + + organizationalUnitName = Organizational Unit (eg, division) + organizationalUnitName_default = Test Unit + + commonName = Common Name (e.g. server FQDN or YOUR name) + commonName_default = Test Name + + emailAddress = Email Address + emailAddress_default = test@test.com + + #################################################################### + [ ca_extensions ] + + subjectKeyIdentifier = hash + authorityKeyIdentifier = keyid:always, issuer + basicConstraints = critical, CA:true + keyUsage = keyCertSign, cRLSign + + #################################################################### + [ signing_policy ] + countryName = optional + stateOrProvinceName = optional + localityName = optional + organizationName = optional + organizationalUnitName = optional + commonName = supplied + emailAddress = optional + + #################################################################### + [ signing_req ] + subjectKeyIdentifier = hash + authorityKeyIdentifier = keyid,issuer + basicConstraints = CA:FALSE + keyUsage = digitalSignature, keyEncipherment + +Then create a database and serial number file, these will be used to keep track of which certificates were signed with this CA. Both of these are simply text files that reside in the same directory as your CA keys. + + > echo 01 > serial.txt + > touch index.txt + +With these steps done you are now ready to generate your CA that will be used to sign certificates later. + + > openssl req -x509 -config openssl-ca.cnf -newkey rsa:4096 -sha256 -nodes -out cacert.pem -outform PEM + +The CA is simply a public/private key pair and certificate that is signed by itself, and is only intended to sign other certificates. +This keypair should be kept very safe, if someone gains access to it, they can create and sign certificates that will be trusted by your infrastructure, which means they will be able to impersonate anybody when connecting to any service that trusts this CA. +The next step is to add the generated CA to the **clients' truststore** so that the clients can trust this CA: + + > keytool -keystore client.truststore.jks -alias CARoot -import -file ca-cert + +**Note:** If you configure the Kafka brokers to require client authentication by setting ssl.client.auth to be "requested" or "required" in the Kafka brokers config then you must provide a truststore for the Kafka brokers as well and it should have all the CA certificates that clients' keys were signed by. + + > keytool -keystore server.truststore.jks -alias CARoot -import -file ca-cert + +In contrast to the keystore in step 1 that stores each machine's own identity, the truststore of a client stores all the certificates that the client should trust. Importing a certificate into one's truststore also means trusting all certificates that are signed by that certificate. As the analogy above, trusting the government (CA) also means trusting all passports (certificates) that it has issued. This attribute is called the chain of trust, and it is particularly useful when deploying SSL on a large Kafka cluster. You can sign all certificates in the cluster with a single CA, and have all machines share the same truststore that trusts the CA. That way all machines can authenticate all other machines. + 3. #### Signing the certificate + +Then sign it with the CA: + + > openssl ca -config openssl-ca.cnf -policy signing_policy -extensions signing_req -out {server certificate} -infiles {certificate signing request} + +Finally, you need to import both the certificate of the CA and the signed certificate into the keystore: + + > keytool -keystore {keystore} -alias CARoot -import -file {CA certificate} + > keytool -keystore {keystore} -alias localhost -import -file cert-signed + +The definitions of the parameters are the following: + 1. keystore: the location of the keystore + 2. CA certificate: the certificate of the CA + 3. certificate signing request: the csr created with the server key + 4. server certificate: the file to write the signed certificate of the server to +This will leave you with one truststore called _truststore.jks_ \- this can be the same for all clients and brokers and does not contain any sensitive information, so there is no need to secure this. +Additionally you will have one _server.keystore.jks_ file per node which contains that nodes keys, certificate and your CAs certificate, please refer to Configuring Kafka Brokers and Configuring Kafka Clients for information on how to use these files. + +For some tooling assistance on this topic, please check out the [easyRSA](https://github.com/OpenVPN/easy-rsa) project which has extensive scripting in place to help with these steps. + +### SSL key and certificates in PEM format + +From 2.7.0 onwards, SSL key and trust stores can be configured for Kafka brokers and clients directly in the configuration in PEM format. This avoids the need to store separate files on the file system and benefits from password protection features of Kafka configuration. PEM may also be used as the store type for file-based key and trust stores in addition to JKS and PKCS12. To configure PEM key store directly in the broker or client configuration, private key in PEM format should be provided in `ssl.keystore.key` and the certificate chain in PEM format should be provided in `ssl.keystore.certificate.chain`. To configure trust store, trust certificates, e.g. public certificate of CA, should be provided in `ssl.truststore.certificates`. Since PEM is typically stored as multi-line base-64 strings, the configuration value can be included in Kafka configuration as multi-line strings with lines terminating in backslash ('\') for line continuation. + +Store password configs `ssl.keystore.password` and `ssl.truststore.password` are not used for PEM. If private key is encrypted using a password, the key password must be provided in `ssl.key.password`. Private keys may be provided in unencrypted form without a password. In production deployments, configs should be encrypted or externalized using password protection feature in Kafka in this case. Note that the default SSL engine factory has limited capabilities for decryption of encrypted private keys when external tools like OpenSSL are used for encryption. Third party libraries like BouncyCastle may be integrated with a custom `SslEngineFactory` to support a wider range of encrypted private keys. + + 4. #### Common Pitfalls in Production + +The above paragraphs show the process to create your own CA and use it to sign certificates for your cluster. While very useful for sandbox, dev, test, and similar systems, this is usually not the correct process to create certificates for a production cluster in a corporate environment. Enterprises will normally operate their own CA and users can send in CSRs to be signed with this CA, which has the benefit of users not being responsible to keep the CA secure as well as a central authority that everybody can trust. However it also takes away a lot of control over the process of signing certificates from the user. Quite often the persons operating corporate CAs will apply tight restrictions on certificates that can cause issues when trying to use these certificates with Kafka. + 1. **[Extended Key Usage](https://tools.ietf.org/html/rfc5280#section-4.2.1.12)** +Certificates may contain an extension field that controls the purpose for which the certificate can be used. If this field is empty, there are no restrictions on the usage, but if any usage is specified in here, valid SSL implementations have to enforce these usages. +Relevant usages for Kafka are: + * Client authentication + * Server authentication +Kafka brokers need both these usages to be allowed, as for intra-cluster communication every broker will behave as both the client and the server towards other brokers. It is not uncommon for corporate CAs to have a signing profile for webservers and use this for Kafka as well, which will only contain the _serverAuth_ usage value and cause the SSL handshake to fail. + 2. **Intermediate Certificates** +Corporate Root CAs are often kept offline for security reasons. To enable day-to-day usage, so called intermediate CAs are created, which are then used to sign the final certificates. When importing a certificate into the keystore that was signed by an intermediate CA it is necessary to provide the entire chain of trust up to the root CA. This can be done by simply _cat_ ing the certificate files into one combined certificate file and then importing this with keytool. + 3. **Failure to copy extension fields** +CA operators are often hesitant to copy and requested extension fields from CSRs and prefer to specify these themselves as this makes it harder for a malicious party to obtain certificates with potentially misleading or fraudulent values. It is advisable to double check signed certificates, whether these contain all requested SAN fields to enable proper hostname verification. The following command can be used to print certificate details to the console, which should be compared with what was originally requested: + + > openssl x509 -in certificate.crt -text -noout + + 5. #### Configuring Kafka Brokers + +If SSL is not enabled for inter-broker communication (see below for how to enable it), both PLAINTEXT and SSL ports will be necessary. + + listeners=PLAINTEXT://host.name:port,SSL://host.name:port + +Following SSL configs are needed on the broker side + + ssl.keystore.location=/var/private/ssl/server.keystore.jks + ssl.keystore.password=test1234 + ssl.key.password=test1234 + ssl.truststore.location=/var/private/ssl/server.truststore.jks + ssl.truststore.password=test1234 + +Note: ssl.truststore.password is technically optional but highly recommended. If a password is not set access to the truststore is still available, but integrity checking is disabled. Optional settings that are worth considering: + 1. ssl.client.auth=none ("required" => client authentication is required, "requested" => client authentication is requested and client without certs can still connect. The usage of "requested" is discouraged as it provides a false sense of security and misconfigured clients will still connect successfully.) + 2. ssl.cipher.suites (Optional). A cipher suite is a named combination of authentication, encryption, MAC and key exchange algorithm used to negotiate the security settings for a network connection using TLS or SSL network protocol. (Default is an empty list) + 3. ssl.enabled.protocols=TLSv1.2,TLSv1.1,TLSv1 (list out the SSL protocols that you are going to accept from clients. Do note that SSL is deprecated in favor of TLS and using SSL in production is not recommended) + 4. ssl.keystore.type=JKS + 5. ssl.truststore.type=JKS + 6. ssl.secure.random.implementation=SHA1PRNG +If you want to enable SSL for inter-broker communication, add the following to the server.properties file (it defaults to PLAINTEXT) + + security.inter.broker.protocol=SSL + +Due to import regulations in some countries, the Oracle implementation limits the strength of cryptographic algorithms available by default. If stronger algorithms are needed (for example, AES with 256-bit keys), the [JCE Unlimited Strength Jurisdiction Policy Files](https://www.oracle.com/technetwork/java/javase/downloads/index.html) must be obtained and installed in the JDK/JRE. See the [JCA Providers Documentation](https://docs.oracle.com/javase/8/docs/technotes/guides/security/SunProviders.html) for more information. + +The JRE/JDK will have a default pseudo-random number generator (PRNG) that is used for cryptography operations, so it is not required to configure the implementation used with the `ssl.secure.random.implementation`. However, there are performance issues with some implementations (notably, the default chosen on Linux systems, `NativePRNG`, utilizes a global lock). In cases where performance of SSL connections becomes an issue, consider explicitly setting the implementation to be used. The `SHA1PRNG` implementation is non-blocking, and has shown very good performance characteristics under heavy load (50 MB/sec of produced messages, plus replication traffic, per-broker). + +Once you start the broker you should be able to see in the server.log + + with addresses: PLAINTEXT -> EndPoint(192.168.64.1,9092,PLAINTEXT),SSL -> EndPoint(192.168.64.1,9093,SSL) + +To check quickly if the server keystore and truststore are setup properly you can run the following command + + > openssl s_client -debug -connect localhost:9093 -tls1 + +(Note: TLSv1 should be listed under ssl.enabled.protocols) +In the output of this command you should see server's certificate: + + -----BEGIN CERTIFICATE----- + {variable sized random bytes} + -----END CERTIFICATE----- + subject=/C=US/ST=CA/L=Santa Clara/O=org/OU=org/CN=Sriharsha Chintalapani + issuer=/C=US/ST=CA/L=Santa Clara/O=org/OU=org/CN=kafka/emailAddress=test@test.com + +If the certificate does not show up or if there are any other error messages then your keystore is not setup properly. + 6. #### Configuring Kafka Clients + +SSL is supported only for the new Kafka Producer and Consumer, the older API is not supported. The configs for SSL will be the same for both producer and consumer. +If client authentication is not required in the broker, then the following is a minimal configuration example: + + security.protocol=SSL + ssl.truststore.location=/var/private/ssl/client.truststore.jks + ssl.truststore.password=test1234 + +Note: ssl.truststore.password is technically optional but highly recommended. If a password is not set access to the truststore is still available, but integrity checking is disabled. If client authentication is required, then a keystore must be created like in step 1 and the following must also be configured: + + ssl.keystore.location=/var/private/ssl/client.keystore.jks + ssl.keystore.password=test1234 + ssl.key.password=test1234 + +Other configuration settings that may also be needed depending on our requirements and the broker configuration: + 1. ssl.provider (Optional). The name of the security provider used for SSL connections. Default value is the default security provider of the JVM. + 2. ssl.cipher.suites (Optional). A cipher suite is a named combination of authentication, encryption, MAC and key exchange algorithm used to negotiate the security settings for a network connection using TLS or SSL network protocol. + 3. ssl.enabled.protocols=TLSv1.2,TLSv1.1,TLSv1. It should list at least one of the protocols configured on the broker side + 4. ssl.truststore.type=JKS + 5. ssl.keystore.type=JKS + +Examples using console-producer and console-consumer: + + > kafka-console-producer.sh --bootstrap-server localhost:9093 --topic test --producer.config client-ssl.properties + > kafka-console-consumer.sh --bootstrap-server localhost:9093 --topic test --consumer.config client-ssl.properties + + + diff --git a/docs/security/incorporating-security-features-in-a-running-cluster.md b/docs/security/incorporating-security-features-in-a-running-cluster.md new file mode 100644 index 0000000000000..2ac0f13f6ecb3 --- /dev/null +++ b/docs/security/incorporating-security-features-in-a-running-cluster.md @@ -0,0 +1,74 @@ +--- +title: Incorporating Security Features in a Running Cluster +description: Incorporating Security Features in a Running Cluster +weight: 6 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +You can secure a running cluster via one or more of the supported protocols discussed previously. This is done in phases: + + * Incrementally bounce the cluster nodes to open additional secured port(s). + * Restart clients using the secured rather than PLAINTEXT port (assuming you are securing the client-broker connection). + * Incrementally bounce the cluster again to enable broker-to-broker security (if this is required) + * A final incremental bounce to close the PLAINTEXT port. + + + +The specific steps for configuring SSL and SASL are described in sections 7.3 and 7.4. Follow these steps to enable security for your desired protocol(s). + +The security implementation lets you configure different protocols for both broker-client and broker-broker communication. These must be enabled in separate bounces. A PLAINTEXT port must be left open throughout so brokers and/or clients can continue to communicate. + +When performing an incremental bounce stop the brokers cleanly via a SIGTERM. It's also good practice to wait for restarted replicas to return to the ISR list before moving onto the next node. + +As an example, say we wish to encrypt both broker-client and broker-broker communication with SSL. In the first incremental bounce, an SSL port is opened on each node: + + + listeners=PLAINTEXT://broker1:9091,SSL://broker1:9092 + +We then restart the clients, changing their config to point at the newly opened, secured port: + + + bootstrap.servers = [broker1:9092,...] + security.protocol = SSL + ...etc + +In the second incremental server bounce we instruct Kafka to use SSL as the broker-broker protocol (which will use the same SSL port): + + + listeners=PLAINTEXT://broker1:9091,SSL://broker1:9092 + security.inter.broker.protocol=SSL + +In the final bounce we secure the cluster by closing the PLAINTEXT port: + + + listeners=SSL://broker1:9092 + security.inter.broker.protocol=SSL + +Alternatively we might choose to open multiple ports so that different protocols can be used for broker-broker and broker-client communication. Say we wished to use SSL encryption throughout (i.e. for broker-broker and broker-client communication) but we'd like to add SASL authentication to the broker-client connection also. We would achieve this by opening two additional ports during the first bounce: + + + listeners=PLAINTEXT://broker1:9091,SSL://broker1:9092,SASL_SSL://broker1:9093 + +We would then restart the clients, changing their config to point at the newly opened, SASL & SSL secured port: + + + bootstrap.servers = [broker1:9093,...] + security.protocol = SASL_SSL + ...etc + +The second server bounce would switch the cluster to use encrypted broker-broker communication via the SSL port we previously opened on port 9092: + + + listeners=PLAINTEXT://broker1:9091,SSL://broker1:9092,SASL_SSL://broker1:9093 + security.inter.broker.protocol=SSL + +The final bounce secures the cluster by closing the PLAINTEXT port. + + + listeners=SSL://broker1:9092,SASL_SSL://broker1:9093 + security.inter.broker.protocol=SSL + +ZooKeeper can be secured independently of the Kafka cluster. The steps for doing this are covered in section 7.7.2. diff --git a/docs/security/listener-configuration.md b/docs/security/listener-configuration.md new file mode 100644 index 0000000000000..4b3cb50b851a3 --- /dev/null +++ b/docs/security/listener-configuration.md @@ -0,0 +1,80 @@ +--- +title: Listener Configuration +description: Listener Configuration +weight: 2 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +In order to secure a Kafka cluster, it is necessary to secure the channels that are used to communicate with the servers. Each server must define the set of listeners that are used to receive requests from clients as well as other servers. Each listener may be configured to authenticate clients using various mechanisms and to ensure traffic between the server and the client is encrypted. This section provides a primer for the configuration of listeners. + +Kafka servers support listening for connections on multiple ports. This is configured through the `listeners` property in the server configuration, which accepts a comma-separated list of the listeners to enable. At least one listener must be defined on each server. The format of each listener defined in `listeners` is given below: + + + {LISTENER_NAME}://{hostname}:{port} + +The `LISTENER_NAME` is usually a descriptive name which defines the purpose of the listener. For example, many configurations use a separate listener for client traffic, so they might refer to the corresponding listener as `CLIENT` in the configuration: + + + listeners=CLIENT://localhost:9092 + +The security protocol of each listener is defined in a separate configuration: `listener.security.protocol.map`. The value is a comma-separated list of each listener mapped to its security protocol. For example, the follow value configuration specifies that the `CLIENT` listener will use SSL while the `BROKER` listener will use plaintext. + + + listener.security.protocol.map=CLIENT:SSL,BROKER:PLAINTEXT + +Possible options (case-insensitive) for the security protocol are given below: + + 1. PLAINTEXT + 2. SSL + 3. SASL_PLAINTEXT + 4. SASL_SSL + + + +The plaintext protocol provides no security and does not require any additional configuration. In the following sections, this document covers how to configure the remaining protocols. + +If each required listener uses a separate security protocol, it is also possible to use the security protocol name as the listener name in `listeners`. Using the example above, we could skip the definition of the `CLIENT` and `BROKER` listeners using the following definition: + + + listeners=SSL://localhost:9092,PLAINTEXT://localhost:9093 + +However, we recommend users to provide explicit names for the listeners since it makes the intended usage of each listener clearer. + +Among the listeners in this list, it is possible to declare the listener to be used for inter-broker communication by setting the `inter.broker.listener.name` configuration to the name of the listener. The primary purpose of the inter-broker listener is partition replication. If not defined, then the inter-broker listener is determined by the security protocol defined by `security.inter.broker.protocol`, which defaults to `PLAINTEXT`. + +For legacy clusters which rely on Zookeeper to store cluster metadata, it is possible to declare a separate listener to be used for metadata propagation from the active controller to the brokers. This is defined by `control.plane.listener.name`. The active controller will use this listener when it needs to push metadata updates to the brokers in the cluster. The benefit of using a control plane listener is that it uses a separate processing thread, which makes it less likely for application traffic to impede timely propagation of metadata changes (such as partition leader and ISR updates). Note that the default value is null, which means that the controller will use the same listener defined by `inter.broker.listener` + +In a KRaft cluster, a broker is any server which has the `broker` role enabled in `process.roles` and a controller is any server which has the `controller` role enabled. Listener configuration depends on the role. The listener defined by `inter.broker.listener.name` is used exclusively for requests between brokers. Controllers, on the other hand, must use separate listener which is defined by the `controller.listener.names` configuration. This cannot be set to the same value as the inter-broker listener. + +Controllers receive requests both from other controllers and from brokers. For this reason, even if a server does not have the `controller` role enabled (i.e. it is just a broker), it must still define the controller listener along with any security properties that are needed to configure it. For example, we might use the following configuration on a standalone broker: + + + process.roles=broker + listeners=BROKER://localhost:9092 + inter.broker.listener.name=BROKER + controller.quorum.voters=0@localhost:9093 + controller.listener.names=CONTROLLER + listener.security.protocol.map=BROKER:SASL_SSL,CONTROLLER:SASL_SSL + +The controller listener is still configured in this example to use the `SASL_SSL` security protocol, but it is not included in `listeners` since the broker does not expose the controller listener itself. The port that will be used in this case comes from the `controller.quorum.voters` configuration, which defines the complete list of controllers. + +For KRaft servers which have both the broker and controller role enabled, the configuration is similar. The only difference is that the controller listener must be included in `listeners`: + + + process.roles=broker,controller + listeners=BROKER://localhost:9092,CONTROLLER://localhost:9093 + inter.broker.listener.name=BROKER + controller.quorum.voters=0@localhost:9093 + controller.listener.names=CONTROLLER + listener.security.protocol.map=BROKER:SASL_SSL,CONTROLLER:SASL_SSL + +It is a requirement for the port defined in `controller.quorum.voters` to exactly match one of the exposed controller listeners. For example, here the `CONTROLLER` listener is bound to port 9093. The connection string defined by `controller.quorum.voters` must then also use port 9093, as it does here. + +The controller will accept requests on all listeners defined by `controller.listener.names`. Typically there would be just one controller listener, but it is possible to have more. For example, this provides a way to change the active listener from one port or security protocol to another through a roll of the cluster (one roll to expose the new listener, and one roll to remove the old listener). When multiple controller listeners are defined, the first one in the list will be used for outbound requests. + +It is conventional in Kafka to use a separate listener for clients. This allows the inter-cluster listeners to be isolated at the network level. In the case of the controller listener in KRaft, the listener should be isolated since clients do not work with it anyway. Clients are expected to connect to any other listener configured on a broker. Any requests that are bound for the controller will be forwarded as described below + +In the following section, this document covers how to enable SSL on a listener for encryption as well as authentication. The subsequent section will then cover additional authentication mechanisms using SASL. diff --git a/docs/security/security-overview.md b/docs/security/security-overview.md new file mode 100644 index 0000000000000..cfd4efd6fa0a1 --- /dev/null +++ b/docs/security/security-overview.md @@ -0,0 +1,23 @@ +--- +title: Security Overview +description: Security Overview +weight: 1 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +In release 0.9.0.0, the Kafka community added a number of features that, used either separately or together, increases security in a Kafka cluster. The following security measures are currently supported: + + 1. Authentication of connections to brokers from clients (producers and consumers), other brokers and tools, using either SSL or SASL. Kafka supports the following SASL mechanisms: + * SASL/GSSAPI (Kerberos) - starting at version 0.9.0.0 + * SASL/PLAIN - starting at version 0.10.0.0 + * SASL/SCRAM-SHA-256 and SASL/SCRAM-SHA-512 - starting at version 0.10.2.0 + * SASL/OAUTHBEARER - starting at version 2.0 + 2. Authentication of connections from brokers to ZooKeeper + 3. Encryption of data transferred between brokers and clients, between brokers, or between brokers and tools using SSL (Note that there is a performance degradation when SSL is enabled, the magnitude of which depends on the CPU type and the JVM implementation.) + 4. Authorization of read / write operations by clients + 5. Authorization is pluggable and integration with external authorization services is supported + +It's worth noting that security is optional - non-secured clusters are supported, as well as a mix of authenticated, unauthenticated, encrypted and non-encrypted clients. The guides below explain how to configure and use the security features in both clients and brokers. diff --git a/docs/security/zookeeper-authentication.md b/docs/security/zookeeper-authentication.md new file mode 100644 index 0000000000000..bd408ef53dd1f --- /dev/null +++ b/docs/security/zookeeper-authentication.md @@ -0,0 +1,123 @@ +--- +title: ZooKeeper Authentication +description: ZooKeeper Authentication +weight: 7 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +ZooKeeper supports mutual TLS (mTLS) authentication beginning with the 3.5.x versions. Kafka supports authenticating to ZooKeeper with SASL and mTLS -- either individually or both together -- beginning with version 2.5. See [KIP-515: Enable ZK client to use the new TLS supported authentication](https://cwiki.apache.org/confluence/display/KAFKA/KIP-515%3A+Enable+ZK+client+to+use+the+new+TLS+supported+authentication) for more details. + +When using mTLS alone, every broker and any CLI tools (such as the ZooKeeper Security Migration Tool) should identify itself with the same Distinguished Name (DN) because it is the DN that is ACL'ed. This can be changed as described below, but it involves writing and deploying a custom ZooKeeper authentication provider. Generally each certificate should have the same DN but a different Subject Alternative Name (SAN) so that hostname verification of the brokers and any CLI tools by ZooKeeper will succeed. + +When using SASL authentication to ZooKeeper together with mTLS, both the SASL identity and either the DN that created the znode (i.e. the creating broker's certificate) or the DN of the Security Migration Tool (if migration was performed after the znode was created) will be ACL'ed, and all brokers and CLI tools will be authorized even if they all use different DNs because they will all use the same ACL'ed SASL identity. It is only when using mTLS authentication alone that all the DNs must match (and SANs become critical -- again, in the absence of writing and deploying a custom ZooKeeper authentication provider as described below). + +Use the broker properties file to set TLS configs for brokers as described below. + +Use the `--zk-tls-config-file ` option to set TLS configs in the Zookeeper Security Migration Tool. The `kafka-acls.sh` and `kafka-configs.sh` CLI tools also support the `--zk-tls-config-file ` option. + +Use the `-zk-tls-config-file ` option (note the single-dash rather than double-dash) to set TLS configs for the `zookeeper-shell.sh` CLI tool. + +## New clusters + +### ZooKeeper SASL Authentication + +To enable ZooKeeper SASL authentication on brokers, there are two necessary steps: + + 1. Create a JAAS login file and set the appropriate system property to point to it as described above + 2. Set the configuration property `zookeeper.set.acl` in each broker to true + +The metadata stored in ZooKeeper for the Kafka cluster is world-readable, but can only be modified by the brokers. The rationale behind this decision is that the data stored in ZooKeeper is not sensitive, but inappropriate manipulation of that data can cause cluster disruption. We also recommend limiting the access to ZooKeeper via network segmentation (only brokers and some admin tools need access to ZooKeeper). + +### ZooKeeper Mutual TLS Authentication + +ZooKeeper mTLS authentication can be enabled with or without SASL authentication. As mentioned above, when using mTLS alone, every broker and any CLI tools (such as the ZooKeeper Security Migration Tool) must generally identify itself with the same Distinguished Name (DN) because it is the DN that is ACL'ed, which means each certificate should have an appropriate Subject Alternative Name (SAN) so that hostname verification of the brokers and any CLI tool by ZooKeeper will succeed. + +It is possible to use something other than the DN for the identity of mTLS clients by writing a class that extends `org.apache.zookeeper.server.auth.X509AuthenticationProvider` and overrides the method `protected String getClientId(X509Certificate clientCert)`. Choose a scheme name and set `authProvider.[scheme]` in ZooKeeper to be the fully-qualified class name of the custom implementation; then set `ssl.authProvider=[scheme]` to use it. + +Here is a sample (partial) ZooKeeper configuration for enabling TLS authentication. These configurations are described in the [ZooKeeper Admin Guide](https://zookeeper.apache.org/doc/r3.5.7/zookeeperAdmin.html#sc_authOptions). + + + secureClientPort=2182 + serverCnxnFactory=org.apache.zookeeper.server.NettyServerCnxnFactory + authProvider.x509=org.apache.zookeeper.server.auth.X509AuthenticationProvider + ssl.keyStore.location=/path/to/zk/keystore.jks + ssl.keyStore.password=zk-ks-passwd + ssl.trustStore.location=/path/to/zk/truststore.jks + ssl.trustStore.password=zk-ts-passwd + +**IMPORTANT** : ZooKeeper does not support setting the key password in the ZooKeeper server keystore to a value different from the keystore password itself. Be sure to set the key password to be the same as the keystore password. + +Here is a sample (partial) Kafka Broker configuration for connecting to ZooKeeper with mTLS authentication. These configurations are described above in Broker Configs. + + + # connect to the ZooKeeper port configured for TLS + zookeeper.connect=zk1:2182,zk2:2182,zk3:2182 + # required to use TLS to ZooKeeper (default is false) + zookeeper.ssl.client.enable=true + # required to use TLS to ZooKeeper + zookeeper.clientCnxnSocket=org.apache.zookeeper.ClientCnxnSocketNetty + # define key/trust stores to use TLS to ZooKeeper; ignored unless zookeeper.ssl.client.enable=true + zookeeper.ssl.keystore.location=/path/to/kafka/keystore.jks + zookeeper.ssl.keystore.password=kafka-ks-passwd + zookeeper.ssl.truststore.location=/path/to/kafka/truststore.jks + zookeeper.ssl.truststore.password=kafka-ts-passwd + # tell broker to create ACLs on znodes + zookeeper.set.acl=true + +**IMPORTANT** : ZooKeeper does not support setting the key password in the ZooKeeper client (i.e. broker) keystore to a value different from the keystore password itself. Be sure to set the key password to be the same as the keystore password. + +## Migrating clusters + +If you are running a version of Kafka that does not support security or simply with security disabled, and you want to make the cluster secure, then you need to execute the following steps to enable ZooKeeper authentication with minimal disruption to your operations: + + 1. Enable SASL and/or mTLS authentication on ZooKeeper. If enabling mTLS, you would now have both a non-TLS port and a TLS port, like this: + + clientPort=2181 + secureClientPort=2182 + serverCnxnFactory=org.apache.zookeeper.server.NettyServerCnxnFactory + authProvider.x509=org.apache.zookeeper.server.auth.X509AuthenticationProvider + ssl.keyStore.location=/path/to/zk/keystore.jks + ssl.keyStore.password=zk-ks-passwd + ssl.trustStore.location=/path/to/zk/truststore.jks + ssl.trustStore.password=zk-ts-passwd + + 2. Perform a rolling restart of brokers setting the JAAS login file and/or defining ZooKeeper mutual TLS configurations (including connecting to the TLS-enabled ZooKeeper port) as required, which enables brokers to authenticate to ZooKeeper. At the end of the rolling restart, brokers are able to manipulate znodes with strict ACLs, but they will not create znodes with those ACLs + 3. If you enabled mTLS, disable the non-TLS port in ZooKeeper + 4. Perform a second rolling restart of brokers, this time setting the configuration parameter `zookeeper.set.acl` to true, which enables the use of secure ACLs when creating znodes + 5. Execute the ZkSecurityMigrator tool. To execute the tool, there is this script: `bin/zookeeper-security-migration.sh` with `zookeeper.acl` set to secure. This tool traverses the corresponding sub-trees changing the ACLs of the znodes. Use the `--zk-tls-config-file ` option if you enable mTLS. + + + +It is also possible to turn off authentication in a secure cluster. To do it, follow these steps: + + 1. Perform a rolling restart of brokers setting the JAAS login file and/or defining ZooKeeper mutual TLS configurations, which enables brokers to authenticate, but setting `zookeeper.set.acl` to false. At the end of the rolling restart, brokers stop creating znodes with secure ACLs, but are still able to authenticate and manipulate all znodes + 2. Execute the ZkSecurityMigrator tool. To execute the tool, run this script `bin/zookeeper-security-migration.sh` with `zookeeper.acl` set to unsecure. This tool traverses the corresponding sub-trees changing the ACLs of the znodes. Use the `--zk-tls-config-file ` option if you need to set TLS configuration. + 3. If you are disabling mTLS, enable the non-TLS port in ZooKeeper + 4. Perform a second rolling restart of brokers, this time omitting the system property that sets the JAAS login file and/or removing ZooKeeper mutual TLS configuration (including connecting to the non-TLS-enabled ZooKeeper port) as required + 5. If you are disabling mTLS, disable the TLS port in ZooKeeper + +Here is an example of how to run the migration tool: + + + > bin/zookeeper-security-migration.sh --zookeeper.acl=secure --zookeeper.connect=localhost:2181 + +Run this to see the full list of parameters: + + + > bin/zookeeper-security-migration.sh --help + +## Migrating the ZooKeeper ensemble + +It is also necessary to enable SASL and/or mTLS authentication on the ZooKeeper ensemble. To do it, we need to perform a rolling restart of the server and set a few properties. See above for mTLS information. Please refer to the ZooKeeper documentation for more detail: + + 1. [Apache ZooKeeper documentation](https://zookeeper.apache.org/doc/r3.5.7/zookeeperProgrammers.html#sc_ZooKeeperAccessControl) + 2. [Apache ZooKeeper wiki](https://cwiki.apache.org/confluence/display/ZOOKEEPER/Zookeeper+and+SASL) + + + +## ZooKeeper Quorum Mutual TLS Authentication + +It is possible to enable mTLS authentication between the ZooKeeper servers themselves. Please refer to the [ZooKeeper documentation](https://zookeeper.apache.org/doc/r3.5.7/zookeeperAdmin.html#Quorum+TLS) for more detail. diff --git a/docs/security/zookeeper-encryption.md b/docs/security/zookeeper-encryption.md new file mode 100644 index 0000000000000..d36ca1a2046a6 --- /dev/null +++ b/docs/security/zookeeper-encryption.md @@ -0,0 +1,25 @@ +--- +title: ZooKeeper Encryption +description: ZooKeeper Encryption +weight: 8 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +ZooKeeper connections that use mutual TLS are encrypted. Beginning with ZooKeeper version 3.5.7 (the version shipped with Kafka version 2.5) ZooKeeper supports a sever-side config `ssl.clientAuth` (case-insensitively: `want`/`need`/`none` are the valid options, the default is `need`), and setting this value to `none` in ZooKeeper allows clients to connect via a TLS-encrypted connection without presenting their own certificate. Here is a sample (partial) Kafka Broker configuration for connecting to ZooKeeper with just TLS encryption. These configurations are described above in Broker Configs. + + + # connect to the ZooKeeper port configured for TLS + zookeeper.connect=zk1:2182,zk2:2182,zk3:2182 + # required to use TLS to ZooKeeper (default is false) + zookeeper.ssl.client.enable=true + # required to use TLS to ZooKeeper + zookeeper.clientCnxnSocket=org.apache.zookeeper.ClientCnxnSocketNetty + # define trust stores to use TLS to ZooKeeper; ignored unless zookeeper.ssl.client.enable=true + # no need to set keystore information assuming ssl.clientAuth=none on ZooKeeper + zookeeper.ssl.truststore.location=/path/to/kafka/truststore.jks + zookeeper.ssl.truststore.password=kafka-ts-passwd + # tell broker to create ACLs on znodes (if using SASL authentication, otherwise do not set this) + zookeeper.set.acl=true diff --git a/docs/streams/_index.md b/docs/streams/_index.md new file mode 100644 index 0000000000000..f68d72a5c1f49 --- /dev/null +++ b/docs/streams/_index.md @@ -0,0 +1,10 @@ +--- +title: Kafka Streams +description: +weight: 9 +tags: ['kafka', 'docs', 'streams'] +aliases: +keywords: +type: docs +--- + diff --git a/docs/streams/architecture.html b/docs/streams/architecture.html deleted file mode 100644 index 1094f16595745..0000000000000 --- a/docs/streams/architecture.html +++ /dev/null @@ -1,206 +0,0 @@ - - - - - - - - -
- - -
- - diff --git a/docs/streams/architecture.md b/docs/streams/architecture.md new file mode 100644 index 0000000000000..415eb3c80749e --- /dev/null +++ b/docs/streams/architecture.md @@ -0,0 +1,81 @@ +--- +title: Architecture +description: +weight: 5 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +Kafka Streams simplifies application development by building on the Kafka producer and consumer libraries and leveraging the native capabilities of Kafka to offer data parallelism, distributed coordination, fault tolerance, and operational simplicity. In this section, we describe how Kafka Streams works underneath the covers. + +The picture below shows the anatomy of an application that uses the Kafka Streams library. Let's walk through some details. + +![](/37/images/streams-architecture-overview.jpg) + +## Stream Partitions and Tasks + +The messaging layer of Kafka partitions data for storing and transporting it. Kafka Streams partitions data for processing it. In both cases, this partitioning is what enables data locality, elasticity, scalability, high performance, and fault tolerance. Kafka Streams uses the concepts of **partitions** and **tasks** as logical units of its parallelism model based on Kafka topic partitions. There are close links between Kafka Streams and Kafka in the context of parallelism: + + * Each **stream partition** is a totally ordered sequence of data records and maps to a Kafka **topic partition**. + * A **data record** in the stream maps to a Kafka **message** from that topic. + * The **keys** of data records determine the partitioning of data in both Kafka and Kafka Streams, i.e., how data is routed to specific partitions within topics. + + + +An application's processor topology is scaled by breaking it into multiple tasks. More specifically, Kafka Streams creates a fixed number of tasks based on the input stream partitions for the application, with each task assigned a list of partitions from the input streams (i.e., Kafka topics). The assignment of partitions to tasks never changes so that each task is a fixed unit of parallelism of the application. Tasks can then instantiate their own processor topology based on the assigned partitions; they also maintain a buffer for each of its assigned partitions and process messages one-at-a-time from these record buffers. As a result stream tasks can be processed independently and in parallel without manual intervention. + +Slightly simplified, the maximum parallelism at which your application may run is bounded by the maximum number of stream tasks, which itself is determined by maximum number of partitions of the input topic(s) the application is reading from. For example, if your input topic has 5 partitions, then you can run up to 5 applications instances. These instances will collaboratively process the topic's data. If you run a larger number of app instances than partitions of the input topic, the "excess" app instances will launch but remain idle; however, if one of the busy instances goes down, one of the idle instances will resume the former's work. + +It is important to understand that Kafka Streams is not a resource manager, but a library that "runs" anywhere its stream processing application runs. Multiple instances of the application are executed either on the same machine, or spread across multiple machines and tasks can be distributed automatically by the library to those running application instances. The assignment of partitions to tasks never changes; if an application instance fails, all its assigned tasks will be automatically restarted on other instances and continue to consume from the same stream partitions. + +**NOTE:** Topic partitions are assigned to tasks, and tasks are assigned to all threads over all instances, in a best-effort attempt to trade off load-balancing and stickiness of stateful tasks. For this assignment, Kafka Streams uses the [StreamsPartitionAssignor](https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java) class and doesn't let you change to a different assignor. If you try to use a different assignor, Kafka Streams ignores it. + +The following diagram shows two tasks each assigned with one partition of the input streams. + +![](/37/images/streams-architecture-tasks.jpg) + + +## Threading Model + +Kafka Streams allows the user to configure the number of **threads** that the library can use to parallelize processing within an application instance. Each thread can execute one or more tasks with their processor topologies independently. For example, the following diagram shows one stream thread running two stream tasks. + +![](/37/images/streams-architecture-threads.jpg) + +Starting more stream threads or more instances of the application merely amounts to replicating the topology and having it process a different subset of Kafka partitions, effectively parallelizing processing. It is worth noting that there is no shared state amongst the threads, so no inter-thread coordination is necessary. This makes it very simple to run topologies in parallel across the application instances and threads. The assignment of Kafka topic partitions amongst the various stream threads is transparently handled by Kafka Streams leveraging [Kafka's coordination](https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Client-side+Assignment+Proposal) functionality. + +As we described above, scaling your stream processing application with Kafka Streams is easy: you merely need to start additional instances of your application, and Kafka Streams takes care of distributing partitions amongst tasks that run in the application instances. You can start as many threads of the application as there are input Kafka topic partitions so that, across all running instances of an application, every thread (or rather, the tasks it runs) has at least one input partition to process. + +As of Kafka 2.8 you can scale stream threads much in the same way you can scale your Kafka Stream clients. Simply add or remove stream threads and Kafka Streams will take care of redistributing the partitions. You may also add threads to replace stream threads that have died removing the need to restart clients to recover the number of thread running. + + + + +## Local State Stores + +Kafka Streams provides so-called **state stores** , which can be used by stream processing applications to store and query data, which is an important capability when implementing stateful operations. The [Kafka Streams DSL](/37/streams/developer-guide/dsl-api.html), for example, automatically creates and manages such state stores when you are calling stateful operators such as `join()` or `aggregate()`, or when you are windowing a stream. + +Every stream task in a Kafka Streams application may embed one or more local state stores that can be accessed via APIs to store and query data required for processing. Kafka Streams offers fault-tolerance and automatic recovery for such local state stores. + +The following diagram shows two stream tasks with their dedicated local state stores. + +![](/37/images/streams-architecture-states.jpg) + + +## Fault Tolerance + +Kafka Streams builds on fault-tolerance capabilities integrated natively within Kafka. Kafka partitions are highly available and replicated; so when stream data is persisted to Kafka it is available even if the application fails and needs to re-process it. Tasks in Kafka Streams leverage the fault-tolerance capability offered by the Kafka consumer client to handle failures. If a task runs on a machine that fails, Kafka Streams automatically restarts the task in one of the remaining running instances of the application. + +In addition, Kafka Streams makes sure that the local state stores are robust to failures, too. For each state store, it maintains a replicated changelog Kafka topic in which it tracks any state updates. These changelog topics are partitioned as well so that each local state store instance, and hence the task accessing the store, has its own dedicated changelog topic partition. [Log compaction](/37/#compaction) is enabled on the changelog topics so that old data can be purged safely to prevent the topics from growing indefinitely. If tasks run on a machine that fails and are restarted on another machine, Kafka Streams guarantees to restore their associated state stores to the content before the failure by replaying the corresponding changelog topics prior to resuming the processing on the newly started tasks. As a result, failure handling is completely transparent to the end user. + +Note that the cost of task (re)initialization typically depends primarily on the time for restoring the state by replaying the state stores' associated changelog topics. To minimize this restoration time, users can configure their applications to have **standby replicas** of local states (i.e. fully replicated copies of the state). When a task migration happens, Kafka Streams will assign a task to an application instance where such a standby replica already exists in order to minimize the task (re)initialization cost. See `num.standby.replicas` in the [**Kafka Streams Configs**](/37/#streamsconfigs) section. Starting in 2.6, Kafka Streams will guarantee that a task is only ever assigned to an instance with a fully caught-up local copy of the state, if such an instance exists. Standby tasks will increase the likelihood that a caught-up instance exists in the case of a failure. + +You can also configure standby replicas with rack awareness. When configured, Kafka Streams will attempt to distribute a standby task on a different "rack" than the active one, thus having a faster recovery time when the rack of the active tasks fails. See `rack.aware.assignment.tags` in the [**Kafka Streams Developer Guide**](/37/streams/developer-guide/config-streams.html#rack-aware-assignment-tags) section. + +There is also a client config `client.rack` which can set the rack for a Kafka consumer. If brokers also have their rack set via `broker.rack`, then rack aware task assignment can be enabled via `rack.aware.assignment.strategy` (cf. [**Kafka Streams Developer Guide**](/37/streams/developer-guide/config-streams.html#rack-aware-assignment-strategy)) to compute a task assignment which can reduce cross rack traffic by trying to assign tasks to clients with the same rack. Note that `client.rack` can also be used to distribute standby tasks to different racks from the active ones, which has a similar functionality as `rack.aware.assignment.tags`. Currently, `rack.aware.assignment.tag` takes precedence in distributing standby tasks which means if both configs present, `rack.aware.assignment.tag` will be used for distributing standby tasks on different racks from the active ones because it can configure more tag keys. + + * [Documentation](/documentation) + * [Kafka Streams](/streams) + + diff --git a/docs/streams/core-concepts.html b/docs/streams/core-concepts.html deleted file mode 100644 index d9a2851e2713f..0000000000000 --- a/docs/streams/core-concepts.html +++ /dev/null @@ -1,387 +0,0 @@ - - - - - - - - -
- - -
- - diff --git a/docs/streams/core-concepts.md b/docs/streams/core-concepts.md new file mode 100644 index 0000000000000..8cf3bbbc7b09f --- /dev/null +++ b/docs/streams/core-concepts.md @@ -0,0 +1,158 @@ +--- +title: Core Concepts +description: +weight: 4 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +Kafka Streams is a client library for processing and analyzing data stored in Kafka. It builds upon important stream processing concepts such as properly distinguishing between event time and processing time, windowing support, and simple yet efficient management and real-time querying of application state. + +Kafka Streams has a **low barrier to entry** : You can quickly write and run a small-scale proof-of-concept on a single machine; and you only need to run additional instances of your application on multiple machines to scale up to high-volume production workloads. Kafka Streams transparently handles the load balancing of multiple instances of the same application by leveraging Kafka's parallelism model. + +Some highlights of Kafka Streams: + + * Designed as a **simple and lightweight client library** , which can be easily embedded in any Java application and integrated with any existing packaging, deployment and operational tools that users have for their streaming applications. + * Has **no external dependencies on systems other than Apache Kafka itself** as the internal messaging layer; notably, it uses Kafka's partitioning model to horizontally scale processing while maintaining strong ordering guarantees. + * Supports **fault-tolerant local state** , which enables very fast and efficient stateful operations like windowed joins and aggregations. + * Supports **exactly-once** processing semantics to guarantee that each record will be processed once and only once even when there is a failure on either Streams clients or Kafka brokers in the middle of processing. + * Employs **one-record-at-a-time processing** to achieve millisecond processing latency, and supports **event-time based windowing operations** with out-of-order arrival of records. + * Offers necessary stream processing primitives, along with a **high-level Streams DSL** and a **low-level Processor API**. + + + +We first summarize the key concepts of Kafka Streams. + +## Stream Processing Topology + + * A **stream** is the most important abstraction provided by Kafka Streams: it represents an unbounded, continuously updating data set. A stream is an ordered, replayable, and fault-tolerant sequence of immutable data records, where a **data record** is defined as a key-value pair. + * A **stream processing application** is any program that makes use of the Kafka Streams library. It defines its computational logic through one or more **processor topologies** , where a processor topology is a graph of stream processors (nodes) that are connected by streams (edges). + * A [**stream processor**](/37/streams/developer-guide/processor-api#defining-a-stream-processor) is a node in the processor topology; it represents a processing step to transform data in streams by receiving one input record at a time from its upstream processors in the topology, applying its operation to it, and may subsequently produce one or more output records to its downstream processors. + +There are two special processors in the topology: + * **Source Processor** : A source processor is a special type of stream processor that does not have any upstream processors. It produces an input stream to its topology from one or multiple Kafka topics by consuming records from these topics and forwarding them to its down-stream processors. + * **Sink Processor** : A sink processor is a special type of stream processor that does not have down-stream processors. It sends any received records from its up-stream processors to a specified Kafka topic. + +Note that in normal processor nodes other remote systems can also be accessed while processing the current record. Therefore the processed results can either be streamed back into Kafka or written to an external system. ![](/37/images/streams-architecture-topology.jpg) + +Kafka Streams offers two ways to define the stream processing topology: the [**Kafka Streams DSL**](/37/streams/developer-guide/dsl-api.html) provides the most common data transformation operations such as `map`, `filter`, `join` and `aggregations` out of the box; the lower-level [**Processor API**](/37/streams/developer-guide/processor-api.html) allows developers define and connect custom processors as well as to interact with state stores. + +A processor topology is merely a logical abstraction for your stream processing code. At runtime, the logical topology is instantiated and replicated inside the application for parallel processing (see [**Stream Partitions and Tasks**](/37/streams/architecture#streams_architecture_tasks) for details). + +## Time + +A critical aspect in stream processing is the notion of **time** , and how it is modeled and integrated. For example, some operations such as **windowing** are defined based on time boundaries. + +Common notions of time in streams are: + + * **Event time** \- The point in time when an event or data record occurred, i.e. was originally created "at the source". **Example:** If the event is a geo-location change reported by a GPS sensor in a car, then the associated event-time would be the time when the GPS sensor captured the location change. + * **Processing time** \- The point in time when the event or data record happens to be processed by the stream processing application, i.e. when the record is being consumed. The processing time may be milliseconds, hours, or days etc. later than the original event time. **Example:** Imagine an analytics application that reads and processes the geo-location data reported from car sensors to present it to a fleet management dashboard. Here, processing-time in the analytics application might be milliseconds or seconds (e.g. for real-time pipelines based on Apache Kafka and Kafka Streams) or hours (e.g. for batch pipelines based on Apache Hadoop or Apache Spark) after event-time. + * **Ingestion time** \- The point in time when an event or data record is stored in a topic partition by a Kafka broker. The difference to event time is that this ingestion timestamp is generated when the record is appended to the target topic by the Kafka broker, not when the record is created "at the source". The difference to processing time is that processing time is when the stream processing application processes the record. **For example,** if a record is never processed, there is no notion of processing time for it, but it still has an ingestion time. + + + +The choice between event-time and ingestion-time is actually done through the configuration of Kafka (not Kafka Streams): From Kafka 0.10.x onwards, timestamps are automatically embedded into Kafka messages. Depending on Kafka's configuration these timestamps represent event-time or ingestion-time. The respective Kafka configuration setting can be specified on the broker level or per topic. The default timestamp extractor in Kafka Streams will retrieve these embedded timestamps as-is. Hence, the effective time semantics of your application depend on the effective Kafka configuration for these embedded timestamps. + +Kafka Streams assigns a **timestamp** to every data record via the `TimestampExtractor` interface. These per-record timestamps describe the progress of a stream with regards to time and are leveraged by time-dependent operations such as window operations. As a result, this time will only advance when a new record arrives at the processor. We call this data-driven time the **stream time** of the application to differentiate with the **wall-clock time** when this application is actually executing. Concrete implementations of the `TimestampExtractor` interface will then provide different semantics to the stream time definition. For example retrieving or computing timestamps based on the actual contents of data records such as an embedded timestamp field to provide event time semantics, and returning the current wall-clock time thereby yield processing time semantics to stream time. Developers can thus enforce different notions of time depending on their business needs. + +Finally, whenever a Kafka Streams application writes records to Kafka, then it will also assign timestamps to these new records. The way the timestamps are assigned depends on the context: + + * When new output records are generated via processing some input record, for example, `context.forward()` triggered in the `process()` function call, output record timestamps are inherited from input record timestamps directly. + * When new output records are generated via periodic functions such as `Punctuator#punctuate()`, the output record timestamp is defined as the current internal time (obtained through `context.timestamp()`) of the stream task. + * For aggregations, the timestamp of a result update record will be the maximum timestamp of all input records contributing to the result. + + + +You can change the default behavior in the Processor API by assigning timestamps to output records explicitly when calling `#forward()`. + +For aggregations and joins, timestamps are computed by using the following rules. + + * For joins (stream-stream, table-table) that have left and right input records, the timestamp of the output record is assigned `max(left.ts, right.ts)`. + * For stream-table joins, the output record is assigned the timestamp from the stream record. + * For aggregations, Kafka Streams also computes the `max` timestamp over all records, per key, either globally (for non-windowed) or per-window. + * For stateless operations, the input record timestamp is passed through. For `flatMap` and siblings that emit multiple records, all output records inherit the timestamp from the corresponding input record. + + + +## Duality of Streams and Tables + +When implementing stream processing use cases in practice, you typically need both **streams** and also **databases**. An example use case that is very common in practice is an e-commerce application that enriches an incoming _stream_ of customer transactions with the latest customer information from a _database table_. In other words, streams are everywhere, but databases are everywhere, too. + +Any stream processing technology must therefore provide **first-class support for streams and tables**. Kafka's Streams API provides such functionality through its core abstractions for [streams](/37/streams/developer-guide/dsl-api#streams_concepts_kstream) and [tables](/37/streams/developer-guide/dsl-api#streams_concepts_ktable), which we will talk about in a minute. Now, an interesting observation is that there is actually a **close relationship between streams and tables** , the so-called stream-table duality. And Kafka exploits this duality in many ways: for example, to make your applications [elastic](/37/streams/developer-guide/running-app#elastic-scaling-of-your-application), to support [fault-tolerant stateful processing](/37/streams/architecture#streams_architecture_recovery), or to run [interactive queries](/37/streams/developer-guide/interactive-queries#interactive-queries) against your application's latest processing results. And, beyond its internal usage, the Kafka Streams API also allows developers to exploit this duality in their own applications. + +Before we discuss concepts such as [aggregations](/37/streams/developer-guide/dsl-api#aggregating) in Kafka Streams, we must first introduce **tables** in more detail, and talk about the aforementioned stream-table duality. Essentially, this duality means that a stream can be viewed as a table, and a table can be viewed as a stream. Kafka's log compaction feature, for example, exploits this duality. + +A simple form of a table is a collection of key-value pairs, also called a map or associative array. Such a table may look as follows: + +![](/37/images/streams-table-duality-01.png) The **stream-table duality** describes the close relationship between streams and tables. + + * **Stream as Table** : A stream can be considered a changelog of a table, where each data record in the stream captures a state change of the table. A stream is thus a table in disguise, and it can be easily turned into a "real" table by replaying the changelog from beginning to end to reconstruct the table. Similarly, in a more general analogy, aggregating data records in a stream - such as computing the total number of pageviews by user from a stream of pageview events - will return a table (here with the key and the value being the user and its corresponding pageview count, respectively). + * **Table as Stream** : A table can be considered a snapshot, at a point in time, of the latest value for each key in a stream (a stream's data records are key-value pairs). A table is thus a stream in disguise, and it can be easily turned into a "real" stream by iterating over each key-value entry in the table. + + + +Let's illustrate this with an example. Imagine a table that tracks the total number of pageviews by user (first column of diagram below). Over time, whenever a new pageview event is processed, the state of the table is updated accordingly. Here, the state changes between different points in time - and different revisions of the table - can be represented as a changelog stream (second column). + +![](/37/images/streams-table-duality-02.png) + +Interestingly, because of the stream-table duality, the same stream can be used to reconstruct the original table (third column): + +![](/37/images/streams-table-duality-03.png) + +The same mechanism is used, for example, to replicate databases via change data capture (CDC) and, within Kafka Streams, to replicate its so-called state stores across machines for fault-tolerance. The stream-table duality is such an important concept that Kafka Streams models it explicitly via the KStream, KTable, and GlobalKTable interfaces. + +## Aggregations + +An **aggregation** operation takes one input stream or table, and yields a new table by combining multiple input records into a single output record. Examples of aggregations are computing counts or sum. + +In the `Kafka Streams DSL`, an input stream of an `aggregation` can be a KStream or a KTable, but the output stream will always be a KTable. This allows Kafka Streams to update an aggregate value upon the out-of-order arrival of further records after the value was produced and emitted. When such out-of-order arrival happens, the aggregating KStream or KTable emits a new aggregate value. Because the output is a KTable, the new value is considered to overwrite the old value with the same key in subsequent processing steps. + +## Windowing + +Windowing lets you control how to _group records that have the same key_ for stateful operations such as `aggregations` or `joins` into so-called _windows_. Windows are tracked per record key. + +`Windowing operations` are available in the `Kafka Streams DSL`. When working with windows, you can specify a **grace period** for the window. This grace period controls how long Kafka Streams will wait for **out-of-order** data records for a given window. If a record arrives after the grace period of a window has passed, the record is discarded and will not be processed in that window. Specifically, a record is discarded if its timestamp dictates it belongs to a window, but the current stream time is greater than the end of the window plus the grace period. + +Out-of-order records are always possible in the real world and should be properly accounted for in your applications. It depends on the effective `time semantics ` how out-of-order records are handled. In the case of processing-time, the semantics are "when the record is being processed", which means that the notion of out-of-order records is not applicable as, by definition, no record can be out-of-order. Hence, out-of-order records can only be considered as such for event-time. In both cases, Kafka Streams is able to properly handle out-of-order records. + +## States + +Some stream processing applications don't require state, which means the processing of a message is independent from the processing of all other messages. However, being able to maintain state opens up many possibilities for sophisticated stream processing applications: you can join input streams, or group and aggregate data records. Many such stateful operators are provided by the [**Kafka Streams DSL**](/37/streams/developer-guide/dsl-api.html). + +Kafka Streams provides so-called **state stores** , which can be used by stream processing applications to store and query data. This is an important capability when implementing stateful operations. Every task in Kafka Streams embeds one or more state stores that can be accessed via APIs to store and query data required for processing. These state stores can either be a persistent key-value store, an in-memory hashmap, or another convenient data structure. Kafka Streams offers fault-tolerance and automatic recovery for local state stores. + +Kafka Streams allows direct read-only queries of the state stores by methods, threads, processes or applications external to the stream processing application that created the state stores. This is provided through a feature called **Interactive Queries**. All stores are named and Interactive Queries exposes only the read operations of the underlying implementation. + + + + +# Processing Guarantees + +In stream processing, one of the most frequently asked question is "does my stream processing system guarantee that each record is processed once and only once, even if some failures are encountered in the middle of processing?" Failing to guarantee exactly-once stream processing is a deal-breaker for many applications that cannot tolerate any data-loss or data duplicates, and in that case a batch-oriented framework is usually used in addition to the stream processing pipeline, known as the [Lambda Architecture](https://en.wikipedia.org/wiki/Lambda_architecture). Prior to 0.11.0.0, Kafka only provides at-least-once delivery guarantees and hence any stream processing systems that leverage it as the backend storage could not guarantee end-to-end exactly-once semantics. In fact, even for those stream processing systems that claim to support exactly-once processing, as long as they are reading from / writing to Kafka as the source / sink, their applications cannot actually guarantee that no duplicates will be generated throughout the pipeline. +Since the 0.11.0.0 release, Kafka has added support to allow its producers to send messages to different topic partitions in a [transactional and idempotent manner](https://kafka.apache.org/#semantics), and Kafka Streams has hence added the end-to-end exactly-once processing semantics by leveraging these features. More specifically, it guarantees that for any record read from the source Kafka topics, its processing results will be reflected exactly once in the output Kafka topic as well as in the state stores for stateful operations. Note the key difference between Kafka Streams end-to-end exactly-once guarantee with other stream processing frameworks' claimed guarantees is that Kafka Streams tightly integrates with the underlying Kafka storage system and ensure that commits on the input topic offsets, updates on the state stores, and writes to the output topics will be completed atomically instead of treating Kafka as an external system that may have side-effects. For more information on how this is done inside Kafka Streams, see [KIP-129](https://cwiki.apache.org/confluence/display/KAFKA/KIP-129%3A+Streams+Exactly-Once+Semantics). +As of the 2.6.0 release, Kafka Streams supports an improved implementation of exactly-once processing, named "exactly-once v2", which requires broker version 2.5.0 or newer. This implementation is more efficient, because it reduces client and broker resource utilization, like client threads and used network connections, and it enables higher throughput and improved scalability. As of the 3.0.0 release, the first version of exactly-once has been deprecated. Users are encouraged to use exactly-once v2 for exactly-once processing from now on, and prepare by upgrading their brokers if necessary. For more information on how this is done inside the brokers and Kafka Streams, see [KIP-447](https://cwiki.apache.org/confluence/display/KAFKA/KIP-447%3A+Producer+scalability+for+exactly+once+semantics). +To enable exactly-once semantics when running Kafka Streams applications, set the `processing.guarantee` config value (default value is **at_least_once**) to **StreamsConfig.EXACTLY_ONCE_V2** (requires brokers version 2.5 or newer). For more information, see the [Kafka Streams Configs](/37/streams/developer-guide/config-streams.html) section. + +## Out-of-Order Handling + +Besides the guarantee that each record will be processed exactly-once, another issue that many stream processing applications will face is how to handle [out-of-order data](https://dl.acm.org/citation.cfm?id=3242155) that may impact their business logic. In Kafka Streams, there are two causes that could potentially result in out-of-order data arrivals with respect to their timestamps: + + * Within a topic-partition, a record's timestamp may not be monotonically increasing along with their offsets. Since Kafka Streams will always try to process records within a topic-partition to follow the offset order, it can cause records with larger timestamps (but smaller offsets) to be processed earlier than records with smaller timestamps (but larger offsets) in the same topic-partition. + * Within a [stream task](/37/streams/architecture#streams_architecture_tasks) that may be processing multiple topic-partitions, if users configure the application to not wait for all partitions to contain some buffered data and pick from the partition with the smallest timestamp to process the next record, then later on when some records are fetched for other topic-partitions, their timestamps may be smaller than those processed records fetched from another topic-partition. + + + +For stateless operations, out-of-order data will not impact processing logic since only one record is considered at a time, without looking into the history of past processed records; for stateful operations such as aggregations and joins, however, out-of-order data could cause the processing logic to be incorrect. If users want to handle such out-of-order data, generally they need to allow their applications to wait for longer time while bookkeeping their states during the wait time, i.e. making trade-off decisions between latency, cost, and correctness. In Kafka Streams specifically, users can configure their window operators for windowed aggregations to achieve such trade-offs (details can be found in [**Developer Guide**](/37/streams/developer-guide)). As for Joins, users may use [versioned state stores](/37/streams/developer-guide/dsl-api.html#versioned-state-stores) to address concerns with out-of-order data, but out-of-order data will not be handled by default: + + * For Stream-Stream joins, all three types (inner, outer, left) handle out-of-order records correctly. + * For Stream-Table joins, if not using versioned stores, then out-of-order records are not handled (i.e., Streams applications don't check for out-of-order records and just process all records in offset order), and hence it may produce unpredictable results. With versioned stores, stream-side out-of-order data will be properly handled by performing a timestamp-based lookup in the table. Table-side out-of-order data is still not handled. + * For Table-Table joins, if not using versioned stores, then out-of-order records are not handled (i.e., Streams applications don't check for out-of-order records and just process all records in offset order). However, the join result is a changelog stream and hence will be eventually consistent. With versioned stores, table-table join semantics change from offset-based semantics to [timestamp-based semantics](/37/streams/developer-guide/dsl-api.html#versioned-state-stores) and out-of-order records are handled accordingly. + + + + * [Documentation](/documentation) + * [Kafka Streams](/streams) + + diff --git a/docs/streams/developer-guide/_index.md b/docs/streams/developer-guide/_index.md new file mode 100644 index 0000000000000..8879fea791903 --- /dev/null +++ b/docs/streams/developer-guide/_index.md @@ -0,0 +1,10 @@ +--- +title: Streams Developer Guide +description: +weight: 10 +tags: ['kafka', 'docs', 'streams', 'developer-guide'] +aliases: +keywords: +type: docs +--- + diff --git a/docs/streams/developer-guide/app-reset-tool.html b/docs/streams/developer-guide/app-reset-tool.html deleted file mode 100644 index a877b54f36e18..0000000000000 --- a/docs/streams/developer-guide/app-reset-tool.html +++ /dev/null @@ -1,203 +0,0 @@ - - - - - - - - - - - diff --git a/docs/streams/developer-guide/app-reset-tool.md b/docs/streams/developer-guide/app-reset-tool.md new file mode 100644 index 0000000000000..bff72ca9e2207 --- /dev/null +++ b/docs/streams/developer-guide/app-reset-tool.md @@ -0,0 +1,135 @@ +--- +title: Application Reset Tool +description: +weight: 13 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +You can reset an application and force it to reprocess its data from scratch by using the application reset tool. This can be useful for development and testing, or when fixing bugs. + +The application reset tool handles the Kafka Streams [user topics](manage-topics.html#streams-developer-guide-topics-user) (input, output, and intermediate topics) and [internal topics](manage-topics.html#streams-developer-guide-topics-internal) differently when resetting the application. + +Here's what the application reset tool does for each topic type: + + * Input topics: Reset offsets to specified position (by default to the beginning of the topic). + * Intermediate topics: Skip to the end of the topic, i.e., set the application's committed consumer offsets for all partitions to each partition's `logSize` (for consumer group `application.id`). + * Internal topics: Delete the internal topic (this automatically deletes any committed offsets). + + + +The application reset tool does not: + + * Reset output topics of an application. If any output (or intermediate) topics are consumed by downstream applications, it is your responsibility to adjust those downstream applications as appropriate when you reset the upstream application. + * Reset the local environment of your application instances. It is your responsibility to delete the local state on any machine on which an application instance was run. See the instructions in section Step 2: Reset the local environments of your application instances on how to do this. + + + +Prerequisites + + + * All instances of your application must be stopped. Otherwise, the application may enter an invalid state, crash, or produce incorrect results. You can verify whether the consumer group with ID `application.id` is still active by using `bin/kafka-consumer-groups`. When long session timeout has been configured, active members could take longer to get expired on the broker thus blocking the reset job to complete. Use the `--force` option could remove those left-over members immediately. Make sure to shut down all stream applications when this option is specified to avoid unexpected rebalances. + + * Use this tool with care and double-check its parameters: If you provide wrong parameter values (e.g., typos in `application.id`) or specify parameters inconsistently (e.g., specify the wrong input topics for the application), this tool might invalidate the application's state or even impact other applications, consumer groups, or your Kafka topics. + + * You should manually delete and re-create any intermediate topics before running the application reset tool. This will free up disk space in Kafka brokers. + + * You should delete and recreate intermediate topics before running the application reset tool, unless the following applies: + +> * You have external downstream consumers for the application's intermediate topics. +> * You are in a development environment where manually deleting and re-creating intermediate topics is unnecessary. + + + + +# Step 1: Run the application reset tool + +Invoke the application reset tool from the command line + +Warning! This tool makes irreversible changes to your application. It is strongly recommended that you run this once with `--dry-run` to preview your changes before making them. + + + /bin/kafka-streams-application-reset + +The tool accepts the following parameters: + + + Option (* = required) Description + --------------------- ----------- + * --application-id The Kafka Streams application ID + (application.id). + --bootstrap-server (deprecated) is specified. The server + (s) to connect to. The broker list + string in the form HOST1:PORT1,HOST2: + PORT2. + --bootstrap-servers DEPRECATED: Comma-separated list of + broker urls with format: HOST1:PORT1, + HOST2:PORT2 (default: localhost:9092) + --by-duration Reset offsets to offset by duration from + current timestamp. Format: 'PnDTnHnMnS' + --config-file Property file containing configs to be + passed to admin clients and embedded + consumer. + --dry-run Display the actions that would be + performed without executing the reset + commands. + --from-file Reset offsets to values defined in CSV + file. + --input-topics Comma-separated list of user input + topics. For these topics, the tool will + reset the offset to the earliest + available offset. + --intermediate-topics Comma-separated list of intermediate user + topics (topics used in the through() + method). For these topics, the tool + will skip to the end. + --internal-topics Comma-separated list of internal topics + to delete. Must be a subset of the + internal topics marked for deletion by + the default behaviour (do a dry-run without + this option to view these topics). + --shift-by Reset offsets shifting current offset by + 'n', where 'n' can be positive or + negative + --to-datetime Reset offsets to offset from datetime. + Format: 'YYYY-MM-DDTHH:mm:SS.sss' + --to-earliest Reset offsets to earliest offset. + --to-latest Reset offsets to latest offset. + --to-offset Reset offsets to a specific offset. + --force Force removing members of the consumer group + (intended to remove left-over members if + long session timeout was configured). + +Consider the following as reset-offset scenarios for `input-topics`: + + * by-duration + * from-file + * shift-by + * to-datetime + * to-earliest + * to-latest + * to-offset + + + +Only one of these scenarios can be defined. If not, `to-earliest` will be executed by default + +All the other parameters can be combined as needed. For example, if you want to restart an application from an empty internal state, but not reprocess previous data, simply omit the parameters `--input-topics` and `--intermediate-topics`. + +# Step 2: Reset the local environments of your application instances + +For a complete application reset, you must delete the application's local state directory on any machines where the application instance was run. You must do this before restarting an application instance on the same machine. You can use either of these methods: + + * The API method `KafkaStreams#cleanUp()` in your application code. + * Manually delete the corresponding local state directory (default location: `/tmp/kafka-streams/`). For more information, see [Streams](/37/javadoc/org/apache/kafka/streams/StreamsConfig.html#STATE_DIR_CONFIG) javadocs. + + + + * [Documentation](/documentation) + * [Kafka Streams](/streams) + * [Developer Guide](/streams/developer-guide/) + + diff --git a/docs/streams/developer-guide/config-streams.html b/docs/streams/developer-guide/config-streams.html deleted file mode 100644 index 70dd5ff0c24b5..0000000000000 --- a/docs/streams/developer-guide/config-streams.html +++ /dev/null @@ -1,1224 +0,0 @@ - - - - - - - - - - - diff --git a/docs/streams/developer-guide/config-streams.md b/docs/streams/developer-guide/config-streams.md new file mode 100644 index 0000000000000..ac6555992b9dc --- /dev/null +++ b/docs/streams/developer-guide/config-streams.md @@ -0,0 +1,627 @@ +--- +title: Configuring a Streams Application +description: +weight: 2 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +Kafka and Kafka Streams configuration options must be configured before using Streams. You can configure Kafka Streams by specifying parameters in a `java.util.Properties` instance. + + 1. Create a `java.util.Properties` instance. + + 2. Set the parameters. For example: + + import java.util.Properties; + import org.apache.kafka.streams.StreamsConfig; + + Properties settings = new Properties(); + // Set a few key parameters + settings.put(StreamsConfig.APPLICATION_ID_CONFIG, "my-first-streams-application"); + settings.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka-broker1:9092"); + // Any further settings + settings.put(... , ...); + + + + +# Configuration parameter reference + +This section contains the most common Streams configuration parameters. For a full reference, see the [Streams](/37/javadoc/org/apache/kafka/streams/StreamsConfig.html) Javadocs. + + * Required configuration parameters + * application.id + * bootstrap.servers + * Recommended configuration parameters for resiliency + * acks + * replication.factor + * Optional configuration parameters + * acceptable.recovery.lag + * default.deserialization.exception.handler + * default.key.serde + * default.production.exception.handler + * default.timestamp.extractor + * default.value.serde + * default.windowed.key.serde.inner + * default.windowed.value.serde.inner + * max.task.idle.ms + * max.warmup.replicas + * num.standby.replicas + * num.stream.threads + * probing.rebalance.interval.ms + * processing.guarantee + * rack.aware.assignment.non_overlap_cost + * rack.aware.assignment.strategy + * rack.aware.assignment.tags + * rack.aware.assignment.traffic_cost + * replication.factor + * rocksdb.config.setter + * state.dir + * topology.optimization + * Kafka consumers and producer configuration parameters + * Naming + * Default Values + * enable.auto.commit + + + +## Required configuration parameters + +Here are the required Streams configuration parameters. + +Parameter Name | Importance | Description | Default Value +---|---|---|--- +application.id | Required | An identifier for the stream processing application. Must be unique within the Kafka cluster. | None +bootstrap.servers | Required | A list of host/port pairs to use for establishing the initial connection to the Kafka cluster. | None + +### application.id + +> (Required) The application ID. Each stream processing application must have a unique ID. The same ID must be given to all instances of the application. It is recommended to use only alphanumeric characters, `.` (dot), `-` (hyphen), and `_` (underscore). Examples: `"hello_world"`, `"hello_world-v1.0.0"` +> +> This ID is used in the following places to isolate resources used by the application from others: +> +> * As the default Kafka consumer and producer `client.id` prefix +> * As the Kafka consumer `group.id` for coordination +> * As the name of the subdirectory in the state directory (cf. `state.dir`) +> * As the prefix of internal Kafka topic names +> + +> +> Tip: +> When an application is updated, the `application.id` should be changed unless you want to reuse the existing data in internal topics and state stores. For example, you could embed the version information within `application.id`, as `my-app-v1.0.0` and `my-app-v1.0.2`. + +### bootstrap.servers + +> (Required) The Kafka bootstrap servers. This is the same [setting](http://kafka.apache.org/documentation.html#producerconfigs) that is used by the underlying producer and consumer clients to connect to the Kafka cluster. Example: `"kafka-broker1:9092,kafka-broker2:9092"`. + +## Recommended configuration parameters for resiliency + +There are several Kafka and Kafka Streams configuration options that need to be configured explicitly for resiliency in face of broker failures: + +Parameter Name | Corresponding Client | Default value | Consider setting to +---|---|---|--- +acks | Producer | `acks=1` | `acks=all` +replication.factor (for broker version 2.3 or older)/td> | Streams | `-1` | `3` +min.insync.replicas | Broker | `1` | `2` +num.standby.replicas | Streams | `0` | `1` + +Increasing the replication factor to 3 ensures that the internal Kafka Streams topic can tolerate up to 2 broker failures. Changing the acks setting to "all" guarantees that a record will not be lost as long as one replica is alive. The tradeoff from moving to the default values to the recommended ones is that some performance and more storage space (3x with the replication factor of 3) are sacrificed for more resiliency. + +### acks + +> The number of acknowledgments that the leader must have received before considering a request complete. This controls the durability of records that are sent. The possible values are: +> +> * `acks=0` The producer does not wait for acknowledgment from the server and the record is immediately added to the socket buffer and considered sent. No guarantee can be made that the server has received the record in this case, and the `retries` configuration will not take effect (as the client won't generally know of any failures). The offset returned for each record will always be set to `-1`. +> * `acks=1` The leader writes the record to its local log and responds without waiting for full acknowledgement from all followers. If the leader immediately fails after acknowledging the record, but before the followers have replicated it, then the record will be lost. +> * `acks=all` The leader waits for the full set of in-sync replicas to acknowledge the record. This guarantees that the record will not be lost if there is at least one in-sync replica alive. This is the strongest available guarantee. +> + +> +> For more information, see the [Kafka Producer documentation](https://kafka.apache.org/#producerconfigs). + +### replication.factor + +> See the description here. + +### num.standby.replicas + +> See the description here. + + + Properties streamsSettings = new Properties(); + // for broker version 2.3 or older + //streamsSettings.put(StreamsConfig.REPLICATION_FACTOR_CONFIG, 3); + streamsSettings.put(StreamsConfig.topicPrefix(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG), 2); + streamsSettings.put(StreamsConfig.producerPrefix(ProducerConfig.ACKS_CONFIG), "all"); + streamsSettings.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1); + +## Optional configuration parameters + +Here are the optional [Streams](/37/javadoc/org/apache/kafka/streams/StreamsConfig.html) javadocs, sorted by level of importance: + +> * High: These parameters can have a significant impact on performance. Take care when deciding the values of these parameters. +> * Medium: These parameters can have some impact on performance. Your specific environment will determine how much tuning effort should be focused on these parameters. +> * Low: These parameters have a less general or less significant impact on performance. +> + + +Parameter Name | Importance | Description | Default Value +---|---|---|--- +acceptable.recovery.lag | Medium | The maximum acceptable lag (number of offsets to catch up) for an instance to be considered caught-up and ready for the active task. | 10000 +application.server | Low | A host:port pair pointing to an embedded user defined endpoint that can be used for discovering the locations of state stores within a single Kafka Streams application. The value of this must be different for each instance of the application. | the empty string +buffered.records.per.partition | Low | The maximum number of records to buffer per partition. | 1000 +cache.max.bytes.buffering | Medium | Maximum number of memory bytes to be used for record caches across all threads. | 10485760 bytes +client.id | Medium | An ID string to pass to the server when making requests. (This setting is passed to the consumer/producer clients used internally by Kafka Streams.) | the empty string +commit.interval.ms | Low | The frequency in milliseconds with which to save the position (offsets in source topics) of tasks. | 30000 milliseconds +default.deserialization.exception.handler | Medium | Exception handling class that implements the `DeserializationExceptionHandler` interface. | `LogAndContinueExceptionHandler` +default.key.serde | Medium | Default serializer/deserializer class for record keys, implements the `Serde` interface. Must be set by the user or all serdes must be passed in explicitly (see also default.value.serde). | `null` +default.production.exception.handler | Medium | Exception handling class that implements the `ProductionExceptionHandler` interface. | `DefaultProductionExceptionHandler` +default.timestamp.extractor | Medium | Timestamp extractor class that implements the `TimestampExtractor` interface. | See Timestamp Extractor +default.value.serde | Medium | Default serializer/deserializer class for record values, implements the `Serde` interface. Must be set by the user or all serdes must be passed in explicitly (see also default.key.serde). | `null` +default.windowed.key.serde.inner | Medium | Default serializer/deserializer for the inner class of windowed keys, implementing the `Serde` interface. | null +default.windowed.value.serde.inner | Medium | Default serializer/deserializer for the inner class of windowed values, implementing the `Serde` interface. | null +default.dsl.store | Low | [DEPRECATED] The default state store type used by DSL operators. Deprecated in favor of `dsl.store.suppliers.class` | `ROCKS_DB` +dsl.store.suppliers.class | Low | Defines a default state store implementation to be used by any stateful DSL operator that has not explicitly configured the store implementation type. Must implement the `org.apache.kafka.streams.state.DslStoreSuppliers` interface. | `BuiltInDslStoreSuppliers.RocksDBDslStoreSuppliers` +max.task.idle.ms | Medium | This config controls whether joins and merges may produce out-of-order results. The config value is the maximum amount of time in milliseconds a stream task will stay idle when it is fully caught up on some (but not all) input partitions to wait for producers to send additional records and avoid potential out-of-order record processing across multiple input streams. The default (zero) does not wait for producers to send more records, but it does wait to fetch data that is already present on the brokers. This default means that for records that are already present on the brokers, Streams will process them in timestamp order. Set to -1 to disable idling entirely and process any locally available data, even though doing so may produce out-of-order processing. | 0 milliseconds +max.warmup.replicas | Medium | The maximum number of warmup replicas (extra standbys beyond the configured num.standbys) that can be assigned at once. | 2 +metric.reporters | Low | A list of classes to use as metrics reporters. | the empty list +metrics.num.samples | Low | The number of samples maintained to compute metrics. | 2 +metrics.recording.level | Low | The highest recording level for metrics. | `INFO` +metrics.sample.window.ms | Low | The window of time in milliseconds a metrics sample is computed over. | 30000 milliseconds (30 seconds) +num.standby.replicas | High | The number of standby replicas for each task. | 0 +num.stream.threads | Medium | The number of threads to execute stream processing. | 1 +probing.rebalance.interval.ms | Low | The maximum time in milliseconds to wait before triggering a rebalance to probe for warmup replicas that have sufficiently caught up. | 600000 milliseconds (10 minutes) +processing.guarantee | Medium | The processing mode. Can be either `"at_least_once"` (default) or `"exactly_once_v2"` (for EOS version 2, requires broker version 2.5+). Deprecated config options are `"exactly_once"` (for EOS version 1) and `"exactly_once_beta"` (for EOS version 2, requires broker version 2.5+). | See Processing Guarantee +poll.ms | Low | The amount of time in milliseconds to block waiting for input. | 100 milliseconds +rack.aware.assignment.tags | Medium | List of tag keys used to distribute standby replicas across Kafka Streams clients. When configured, Kafka Streams will make a best-effort to distribute the standby tasks over clients with different tag values. | the empty list +replication.factor | Medium | The replication factor for changelog topics and repartition topics created by the application. The default of `-1` (meaning: use broker default replication factor) requires broker version 2.4 or newer. | -1 +retry.backoff.ms | Medium | The amount of time in milliseconds, before a request is retried. This applies if the `retries` parameter is configured to be greater than 0. | 100 milliseconds +rocksdb.config.setter | Medium | The RocksDB configuration. | +state.cleanup.delay.ms | Low | The amount of time in milliseconds to wait before deleting state when a partition has migrated. | 600000 milliseconds (10 minutes) +state.dir | High | Directory location for state stores. | `/${java.io.tmpdir}/kafka-streams` +task.timeout.ms | Medium | The maximum amount of time in milliseconds a task might stall due to internal errors and retries until an error is raised. For a timeout of `0 ms`, a task would raise an error for the first internal error. For any timeout larger than `0 ms`, a task will retry at least once before an error is raised. | 300000 milliseconds (5 minutes) +topology.optimization | Medium | A configuration telling Kafka Streams if it should optimize the topology and what optimizations to apply. Acceptable values are: `StreamsConfig.NO_OPTIMIZATION` (`none`), `StreamsConfig.OPTIMIZE` (`all`) or a comma separated list of specific optimizations: (`StreamsConfig.REUSE_KTABLE_SOURCE_TOPICS` (`reuse.ktable.source.topics`), `StreamsConfig.MERGE_REPARTITION_TOPICS` (`merge.repartition.topics`)). | ` NO_OPTIMIZATION` +upgrade.from | Medium | The version you are upgrading from during a rolling upgrade. | See Upgrade From +windowstore.changelog.additional.retention.ms | Low | Added to a windows maintainMs to ensure data is not deleted from the log prematurely. Allows for clock drift. | 86400000 milliseconds (1 day) +window.size.ms | Low | Sets window size for the deserializer in order to calculate window end times. | null + +### acceptable.recovery.lag + +> The maximum acceptable lag (total number of offsets to catch up from the changelog) for an instance to be considered caught-up and able to receive an active task. Streams will only assign stateful active tasks to instances whose state stores are within the acceptable recovery lag, if any exist, and assign warmup replicas to restore state in the background for instances that are not yet caught up. Should correspond to a recovery time of well under a minute for a given workload. Must be at least 0. +> +> Note: if you set this to `Long.MAX_VALUE` it effectively disables the warmup replicas and task high availability, allowing Streams to immediately produce a balanced assignment and migrate tasks to a new instance without first warming them up. + +### default.deserialization.exception.handler + +> The default deserialization exception handler allows you to manage record exceptions that fail to deserialize. This can be caused by corrupt data, incorrect serialization logic, or unhandled record types. The implemented exception handler needs to return a `FAIL` or `CONTINUE` depending on the record and the exception thrown. Returning `FAIL` will signal that Streams should shut down and `CONTINUE` will signal that Streams should ignore the issue and continue processing. The following library built-in exception handlers are available: +> +> * [LogAndContinueExceptionHandler](/37/javadoc/org/apache/kafka/streams/errors/LogAndContinueExceptionHandler.html): This handler logs the deserialization exception and then signals the processing pipeline to continue processing more records. This log-and-skip strategy allows Kafka Streams to make progress instead of failing if there are records that fail to deserialize. +> * [LogAndFailExceptionHandler](/37/javadoc/org/apache/kafka/streams/errors/LogAndFailExceptionHandler.html). This handler logs the deserialization exception and then signals the processing pipeline to stop processing more records. +> + +> +> You can also provide your own customized exception handler besides the library provided ones to meet your needs. For example, you can choose to forward corrupt records into a quarantine topic (think: a "dead letter queue") for further processing. To do this, use the Producer API to write a corrupted record directly to the quarantine topic. To be more concrete, you can create a separate `KafkaProducer` object outside the Streams client, and pass in this object as well as the dead letter queue topic name into the `Properties` map, which then can be retrieved from the `configure` function call. The drawback of this approach is that "manual" writes are side effects that are invisible to the Kafka Streams runtime library, so they do not benefit from the end-to-end processing guarantees of the Streams API: +> +> +> public class SendToDeadLetterQueueExceptionHandler implements DeserializationExceptionHandler { +> KafkaProducer dlqProducer; +> String dlqTopic; +> +> @Override +> public DeserializationHandlerResponse handle(final ProcessorContext context, +> final ConsumerRecord record, +> final Exception exception) { +> +> log.warn("Exception caught during Deserialization, sending to the dead queue topic; " + +> "taskId: {}, topic: {}, partition: {}, offset: {}", +> context.taskId(), record.topic(), record.partition(), record.offset(), +> exception); +> +> dlqProducer.send(new ProducerRecord<>(dlqTopic, record.timestamp(), record.key(), record.value(), record.headers())).get(); +> +> return DeserializationHandlerResponse.CONTINUE; +> } +> +> @Override +> public void configure(final Map configs) { +> dlqProducer = .. // get a producer from the configs map +> dlqTopic = .. // get the topic name from the configs map +> } +> } + +### default.production.exception.handler + +> The default production exception handler allows you to manage exceptions triggered when trying to interact with a broker such as attempting to produce a record that is too large. By default, Kafka provides and uses the [DefaultProductionExceptionHandler](/37/javadoc/org/apache/kafka/streams/errors/DefaultProductionExceptionHandler.html) that always fails when these exceptions occur. +> +> Each exception handler can return a `FAIL` or `CONTINUE` depending on the record and the exception thrown. Returning `FAIL` will signal that Streams should shut down and `CONTINUE` will signal that Streams should ignore the issue and continue processing. If you want to provide an exception handler that always ignores records that are too large, you could implement something like the following: +> +> +> import java.util.Properties; +> import org.apache.kafka.streams.StreamsConfig; +> import org.apache.kafka.common.errors.RecordTooLargeException; +> import org.apache.kafka.streams.errors.ProductionExceptionHandler; +> import org.apache.kafka.streams.errors.ProductionExceptionHandler.ProductionExceptionHandlerResponse; +> +> public class IgnoreRecordTooLargeHandler implements ProductionExceptionHandler { +> public void configure(Map config) {} +> +> public ProductionExceptionHandlerResponse handle(final ProducerRecord record, +> final Exception exception) { +> if (exception instanceof RecordTooLargeException) { +> return ProductionExceptionHandlerResponse.CONTINUE; +> } else { +> return ProductionExceptionHandlerResponse.FAIL; +> } +> } +> } +> +> Properties settings = new Properties(); +> +> // other various kafka streams settings, e.g. bootstrap servers, application id, etc +> +> settings.put(StreamsConfig.DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_CONFIG, +> IgnoreRecordTooLargeHandler.class); + +### default.timestamp.extractor + +> A timestamp extractor pulls a timestamp from an instance of [ConsumerRecord](/37/javadoc/org/apache/kafka/clients/consumer/ConsumerRecord.html). Timestamps are used to control the progress of streams. +> +> The default extractor is [FailOnInvalidTimestamp](/37/javadoc/org/apache/kafka/streams/processor/FailOnInvalidTimestamp.html). This extractor retrieves built-in timestamps that are automatically embedded into Kafka messages by the Kafka producer client since [Kafka version 0.10](https://cwiki.apache.org/confluence/display/KAFKA/KIP-32+-+Add+timestamps+to+Kafka+message). Depending on the setting of Kafka's server-side `log.message.timestamp.type` broker and `message.timestamp.type` topic parameters, this extractor provides you with: +> +> * **event-time** processing semantics if `log.message.timestamp.type` is set to `CreateTime` aka "producer time" (which is the default). This represents the time when a Kafka producer sent the original message. If you use Kafka's official producer client, the timestamp represents milliseconds since the epoch. +> * **ingestion-time** processing semantics if `log.message.timestamp.type` is set to `LogAppendTime` aka "broker time". This represents the time when the Kafka broker received the original message, in milliseconds since the epoch. +> + +> +> The `FailOnInvalidTimestamp` extractor throws an exception if a record contains an invalid (i.e. negative) built-in timestamp, because Kafka Streams would not process this record but silently drop it. Invalid built-in timestamps can occur for various reasons: if for example, you consume a topic that is written to by pre-0.10 Kafka producer clients or by third-party producer clients that don't support the new Kafka 0.10 message format yet; another situation where this may happen is after upgrading your Kafka cluster from `0.9` to `0.10`, where all the data that was generated with `0.9` does not include the `0.10` message timestamps. +> +> If you have data with invalid timestamps and want to process it, then there are two alternative extractors available. Both work on built-in timestamps, but handle invalid timestamps differently. +> +> * [LogAndSkipOnInvalidTimestamp](/37/javadoc/org/apache/kafka/streams/processor/LogAndSkipOnInvalidTimestamp.html): This extractor logs a warn message and returns the invalid timestamp to Kafka Streams, which will not process but silently drop the record. This log-and-skip strategy allows Kafka Streams to make progress instead of failing if there are records with an invalid built-in timestamp in your input data. +> * [UsePartitionTimeOnInvalidTimestamp](/37/javadoc/org/apache/kafka/streams/processor/UsePartitionTimeOnInvalidTimestamp.html). This extractor returns the record's built-in timestamp if it is valid (i.e. not negative). If the record does not have a valid built-in timestamps, the extractor returns the previously extracted valid timestamp from a record of the same topic partition as the current record as a timestamp estimation. In case that no timestamp can be estimated, it throws an exception. +> + +> +> Another built-in extractor is [WallclockTimestampExtractor](/37/javadoc/org/apache/kafka/streams/processor/WallclockTimestampExtractor.html). This extractor does not actually "extract" a timestamp from the consumed record but rather returns the current time in milliseconds from the system clock (think: `System.currentTimeMillis()`), which effectively means Streams will operate on the basis of the so-called **processing-time** of events. +> +> You can also provide your own timestamp extractors, for instance to retrieve timestamps embedded in the payload of messages. If you cannot extract a valid timestamp, you can either throw an exception, return a negative timestamp, or estimate a timestamp. Returning a negative timestamp will result in data loss - the corresponding record will not be processed but silently dropped. If you want to estimate a new timestamp, you can use the value provided via `previousTimestamp` (i.e., a Kafka Streams timestamp estimation). Here is an example of a custom `TimestampExtractor` implementation: +> +> +> import org.apache.kafka.clients.consumer.ConsumerRecord; +> import org.apache.kafka.streams.processor.TimestampExtractor; +> +> // Extracts the embedded timestamp of a record (giving you "event-time" semantics). +> public class MyEventTimeExtractor implements TimestampExtractor { +> +> @Override +> public long extract(final ConsumerRecord record, final long previousTimestamp) { +> // `Foo` is your own custom class, which we assume has a method that returns +> // the embedded timestamp (milliseconds since midnight, January 1, 1970 UTC). +> long timestamp = -1; +> final Foo myPojo = (Foo) record.value(); +> if (myPojo != null) { +> timestamp = myPojo.getTimestampInMillis(); +> } +> if (timestamp < 0) { +> // Invalid timestamp! Attempt to estimate a new timestamp, +> // otherwise fall back to wall-clock time (processing-time). +> if (previousTimestamp >= 0) { +> return previousTimestamp; +> } else { +> return System.currentTimeMillis(); +> } +> } +> } +> +> } +> +> You would then define the custom timestamp extractor in your Streams configuration as follows: +> +> +> import java.util.Properties; +> import org.apache.kafka.streams.StreamsConfig; +> +> Properties streamsConfiguration = new Properties(); +> streamsConfiguration.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MyEventTimeExtractor.class); + +### default.key.serde + +> The default Serializer/Deserializer class for record keys, null unless set by user. Serialization and deserialization in Kafka Streams happens whenever data needs to be materialized, for example: +> +> * Whenever data is read from or written to a _Kafka topic_ (e.g., via the `StreamsBuilder#stream()` and `KStream#to()` methods). +> * Whenever data is read from or written to a _state store_. +> + +> +> This is discussed in more detail in [Data types and serialization](datatypes.html#streams-developer-guide-serdes). + +### default.value.serde + +> The default Serializer/Deserializer class for record values, null unless set by user. Serialization and deserialization in Kafka Streams happens whenever data needs to be materialized, for example: +> +> * Whenever data is read from or written to a _Kafka topic_ (e.g., via the `StreamsBuilder#stream()` and `KStream#to()` methods). +> * Whenever data is read from or written to a _state store_. +> + +> +> This is discussed in more detail in [Data types and serialization](datatypes.html#streams-developer-guide-serdes). + +### default.windowed.key.serde.inner + +> The default Serializer/Deserializer class for the inner class of windowed keys. Serialization and deserialization in Kafka Streams happens whenever data needs to be materialized, for example: +> +> * Whenever data is read from or written to a _Kafka topic_ (e.g., via the `StreamsBuilder#stream()` and `KStream#to()` methods). +> * Whenever data is read from or written to a _state store_. +> + +> +> This is discussed in more detail in [Data types and serialization](datatypes.html#streams-developer-guide-serdes). + +### default.windowed.value.serde.inner + +> The default Serializer/Deserializer class for the inner class of windowed values. Serialization and deserialization in Kafka Streams happens happens whenever data needs to be materialized, for example: +> +> * Whenever data is read from or written to a _Kafka topic_ (e.g., via the `StreamsBuilder#stream()` and `KStream#to()` methods). +> * Whenever data is read from or written to a _state store_. +> + +> +> This is discussed in more detail in [Data types and serialization](datatypes.html#streams-developer-guide-serdes). + +### rack.aware.assignment.non_overlap_cost + +> This configuration sets the cost of moving a task from the original assignment computed either by `StickyTaskAssignor` or `HighAvailabilityTaskAssignor`. Together with `rack.aware.assignment.traffic_cost`, they control whether the optimizer favors minimizing cross rack traffic or minimizing the movement of tasks in the existing assignment. If this config is set to a larger value than `rack.aware.assignment.traffic_cost`, the optimizer will try to maintain the existing assignment computed by the task assignor. Note that the optimizer takes the ratio of these two configs into consideration of favoring maintaining existing assignment or minimizing traffic cost. For example, setting `rack.aware.assignment.non_overlap_cost` to 10 and `rack.aware.assignment.traffic_cost` to 1 is more likely to maintain existing assignment than setting `rack.aware.assignment.non_overlap_cost` to 100 and `rack.aware.assignment.traffic_cost` to 50. +> +> The default value is null which means default `non_overlap_cost` in different assignors will be used. In `StickyTaskAssignor`, it has a default value of 10 and `rack.aware.assignment.traffic_cost` has a default value of 1, which means maintaining stickiness is preferred in `StickyTaskAssignor`. In `HighAvailabilityTaskAssignor`, it has a default value of 1 and `rack.aware.assignment.traffic_cost` has a default value of 10, which means minimizing cross rack traffic is preferred in `HighAvailabilityTaskAssignor`. + +### rack.aware.assignment.strategy + +> This configuration sets the strategy Kafka Streams uses for rack aware task assignment so that cross traffic from broker to client can be reduced. This config will only take effect when `broker.rack` is set on the brokers and `client.rack` is set on Kafka Streams side. There are two settings for this config: +> +> * `none`. This is the default value which means rack aware task assignment will be disabled. +> * `min_traffic`. This settings means that the rack aware task assigner will compute an assignment which tries to minimize cross rack traffic. +> * `balance_subtopology`. This settings means that the rack aware task assigner will compute an assignment which will try to balance tasks from same subtopology to different clients and minimize cross rack traffic on top of that. +> + +> +> This config can be used together with rack.aware.assignment.non_overlap_cost and rack.aware.assignment.traffic_cost to balance reducing cross rack traffic and maintaining the existing assignment. + +### rack.aware.assignment.tags + +> This configuration sets a list of tag keys used to distribute standby replicas across Kafka Streams clients. When configured, Kafka Streams will make a best-effort to distribute the standby tasks over clients with different tag values. +> +> Tags for the Kafka Streams clients can be set via `client.tag.` prefix. Example: +> +> +> Client-1 | Client-2 +> _______________________________________________________________________ +> client.tag.zone: eu-central-1a | client.tag.zone: eu-central-1b +> client.tag.cluster: k8s-cluster1 | client.tag.cluster: k8s-cluster1 +> rack.aware.assignment.tags: zone,cluster | rack.aware.assignment.tags: zone,cluster +> +> +> Client-3 | Client-4 +> _______________________________________________________________________ +> client.tag.zone: eu-central-1a | client.tag.zone: eu-central-1b +> client.tag.cluster: k8s-cluster2 | client.tag.cluster: k8s-cluster2 +> rack.aware.assignment.tags: zone,cluster | rack.aware.assignment.tags: zone,cluster +> +> In the above example, we have four Kafka Streams clients across two zones (`eu-central-1a`, `eu-central-1b`) and across two clusters (`k8s-cluster1`, `k8s-cluster2`). For an active task located on `Client-1`, Kafka Streams will allocate a standby task on `Client-4`, since `Client-4` has a different `zone` and a different `cluster` than `Client-1`. + +### rack.aware.assignment.traffic_cost + +> This configuration sets the cost of cross rack traffic. Together with `rack.aware.assignment.non_overlap_cost`, they control whether the optimizer favors minimizing cross rack traffic or minimizing the movement of tasks in the existing assignment. If this config is set to a larger value than `rack.aware.assignment.non_overlap_cost`, the optimizer will try to compute an assignment which minimize the cross rack traffic. Note that the optimizer takes the ratio of these two configs into consideration of favoring maintaining existing assignment or minimizing traffic cost. For example, setting `rack.aware.assignment.traffic_cost` to 10 and `rack.aware.assignment.non_overlap_cost` to 1 is more likely to minimize cross rack traffic than setting `rack.aware.assignment.traffic_cost` to 100 and `rack.aware.assignment.non_overlap_cost` to 50. +> +> The default value is null which means default traffic cost in different assignors will be used. In `StickyTaskAssignor`, it has a default value of 1 and `rack.aware.assignment.non_overlap_cost` has a default value of 10. In `HighAvailabilityTaskAssignor`, it has a default value of 10 and `rack.aware.assignment.non_overlap_cost` has a default value of 1. + +### max.task.idle.ms + +> This configuration controls how long Streams will wait to fetch data in order to provide in-order processing semantics. +> +> When processing a task that has multiple input partitions (as in a join or merge), Streams needs to choose which partition to process the next record from. When all input partitions have locally buffered data, Streams picks the partition whose next record has the lowest timestamp. This has the desirable effect of collating the input partitions in timestamp order, which is generally what you want in a streaming join or merge. However, when Streams does not have any data buffered locally for one of the partitions, it does not know whether the next record for that partition will have a lower or higher timestamp than the remaining partitions' records. +> +> There are two cases to consider: either there is data in that partition on the broker that Streams has not fetched yet, or Streams is fully caught up with that partition on the broker, and the producers simply haven't produced any new records since Streams polled the last batch. +> +> The default value of `0` causes Streams to delay processing a task when it detects that it has no locally buffered data for a partition, but there is data available on the brokers. Specifically, when there is an empty partition in the local buffer, but Streams has a non-zero lag for that partition. However, as soon as Streams catches up to the broker, it will continue processing, even if there is no data in one of the partitions. That is, it will not wait for new data to be _produced_. This default is designed to sacrifice some throughput in exchange for intuitively correct join semantics. +> +> Any config value greater than zero indicates the number of _extra_ milliseconds that Streams will wait if it has a caught-up but empty partition. In other words, this is the amount of time to wait for new data to be produced to the input partitions to ensure in-order processing of data in the event of a slow producer. +> +> The config value of `-1` indicates that Streams will never wait to buffer empty partitions before choosing the next record by timestamp, which achieves maximum throughput at the expense of introducing out-of-order processing. + +### max.warmup.replicas + +> The maximum number of warmup replicas (extra standbys beyond the configured `num.standbys`) that can be assigned at once for the purpose of keeping the task available on one instance while it is warming up on another instance it has been reassigned to. Used to throttle how much extra broker traffic and cluster state can be used for high availability. Increasing this will allow Streams to warm up more tasks at once, speeding up the time for the reassigned warmups to restore sufficient state for them to be transitioned to active tasks. Must be at least 1. +> +> Note that one warmup replica corresponds to one [Stream Task](https://kafka.apache.org/34/streams/architecture#streams_architecture_tasks). Furthermore, note that each warmup task can only be promoted to an active task during a rebalance (normally during a so-called probing rebalance, which occur at a frequency specified by the `probing.rebalance.interval.ms` config). This means that the maximum rate at which active tasks can be migrated from one Kafka Streams instance to another instance can be determined by (`max.warmup.replicas` / `probing.rebalance.interval.ms`). + +### num.standby.replicas + +> The number of standby replicas. Standby replicas are shadow copies of local state stores. Kafka Streams attempts to create the specified number of replicas per store and keep them up to date as long as there are enough instances running. Standby replicas are used to minimize the latency of task failover. A task that was previously running on a failed instance is preferred to restart on an instance that has standby replicas so that the local state store restoration process from its changelog can be minimized. Details about how Kafka Streams makes use of the standby replicas to minimize the cost of resuming tasks on failover can be found in the [State](../architecture.html#streams_architecture_state) section. +> +> Recommendation: +> Increase the number of standbys to 1 to get instant fail-over, i.e., high-availability. Increasing the number of standbys requires more client-side storage space. For example, with 1 standby, 2x space is required. +> +> Note: +> If you enable n standby tasks, you need to provision n+1 `KafkaStreams` instances. + +### num.stream.threads + +> This specifies the number of stream threads in an instance of the Kafka Streams application. The stream processing code runs in these thread. For more information about Kafka Streams threading model, see [Threading Model](../architecture.html#streams_architecture_threads). + +### probing.rebalance.interval.ms + +> The maximum time to wait before triggering a rebalance to probe for warmup replicas that have restored enough to be considered caught up. Streams will only assign stateful active tasks to instances that are caught up and within the acceptable.recovery.lag, if any exist. Probing rebalances are used to query the latest total lag of warmup replicas and transition them to active tasks if ready. They will continue to be triggered as long as there are warmup tasks, and until the assignment is balanced. Must be at least 1 minute. + +### processing.guarantee + +> The processing guarantee that should be used. Possible values are `"at_least_once"` (default) and `"exactly_once_v2"` (for EOS version 2). Deprecated config options are `"exactly_once"` (for EOS alpha), and `"exactly_once_beta"` (for EOS version 2). Using `"exactly_once_v2"` (or the deprecated `"exactly_once_beta"`) requires broker version 2.5 or newer, while using the deprecated `"exactly_once"` requires broker version 0.11.0 or newer. Note that if exactly-once processing is enabled, the default for parameter `commit.interval.ms` changes to 100ms. Additionally, consumers are configured with `isolation.level="read_committed"` and producers are configured with `enable.idempotence=true` per default. Note that by default exactly-once processing requires a cluster of at least three brokers what is the recommended setting for production. For development, you can change this configuration by adjusting broker setting `transaction.state.log.replication.factor` and `transaction.state.log.min.isr` to the number of brokers you want to use. For more details see [Processing Guarantees](../core-concepts#streams_processing_guarantee). +> +> Recommendation: +> While it is technically possible to use EOS with any replication factor, using a replication factor lower than 3 effectively voids EOS. Thus it is strongly recommended to use a replication factor of 3 (together with `min.in.sync.replicas=2`). This recommendation applies to all topics (i.e. `__transaction_state`, `__consumer_offsets`, Kafka Streams internal topics, and user topics). + +### replication.factor + +> This specifies the replication factor of internal topics that Kafka Streams creates when local states are used or a stream is repartitioned for aggregation. Replication is important for fault tolerance. Without replication even a single broker failure may prevent progress of the stream processing application. It is recommended to use a similar replication factor as source topics. +> +> Recommendation: +> Increase the replication factor to 3 to ensure that the internal Kafka Streams topic can tolerate up to 2 broker failures. Note that you will require more storage space as well (3x with the replication factor of 3). + +### rocksdb.config.setter + +> The RocksDB configuration. Kafka Streams uses RocksDB as the default storage engine for persistent stores. To change the default configuration for RocksDB, you can implement `RocksDBConfigSetter` and provide your custom class via [rocksdb.config.setter](/37/javadoc/org/apache/kafka/streams/state/RocksDBConfigSetter.html). +> +> Here is an example that adjusts the memory size consumed by RocksDB. +> +> +> public static class CustomRocksDBConfig implements RocksDBConfigSetter { +> // This object should be a member variable so it can be closed in RocksDBConfigSetter#close. +> private org.rocksdb.Cache cache = new org.rocksdb.LRUCache(16 * 1024L * 1024L); +> +> @Override +> public void setConfig(final String storeName, final Options options, final Map configs) { +> // See #1 below. +> BlockBasedTableConfig tableConfig = (BlockBasedTableConfig) options.tableFormatConfig(); +> tableConfig.setBlockCache(cache); +> // See #2 below. +> tableConfig.setBlockSize(16 * 1024L); +> // See #3 below. +> tableConfig.setCacheIndexAndFilterBlocks(true); +> options.setTableFormatConfig(tableConfig); +> // See #4 below. +> options.setMaxWriteBufferNumber(2); +> } +> +> @Override +> public void close(final String storeName, final Options options) { +> // See #5 below. +> cache.close(); +> } +> } +> +> Properties streamsSettings = new Properties(); +> streamsConfig.put(StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG, CustomRocksDBConfig.class); +> +> Notes for example: +> +> +> 1. `BlockBasedTableConfig tableConfig = (BlockBasedTableConfig) options.tableFormatConfig();` Get a reference to the existing table config rather than create a new one, so you don't accidentally overwrite defaults such as the `BloomFilter`, which is an important optimization. +> 2. `tableConfig.setBlockSize(16 * 1024L);` Modify the default [block size](https://github.com/apache/kafka/blob/2.3/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java#L79) per these instructions from the [RocksDB GitHub](https://github.com/facebook/rocksdb/wiki/Memory-usage-in-RocksDB#indexes-and-filter-blocks). +> 3. `tableConfig.setCacheIndexAndFilterBlocks(true);` Do not let the index and filter blocks grow unbounded. For more information, see the [RocksDB GitHub](https://github.com/facebook/rocksdb/wiki/Block-Cache#caching-index-and-filter-blocks). +> 4. `options.setMaxWriteBufferNumber(2);` See the advanced options in the [RocksDB GitHub](https://github.com/facebook/rocksdb/blob/8dee8cad9ee6b70fd6e1a5989a8156650a70c04f/include/rocksdb/advanced_options.h#L103). +> 5. `cache.close();` To avoid memory leaks, you must close any objects you constructed that extend org.rocksdb.RocksObject. See [RocksJava docs](https://github.com/facebook/rocksdb/wiki/RocksJava-Basics#memory-management) for more details. +> + + + #### state.dir + +> The state directory. Kafka Streams persists local states under the state directory. Each application has a subdirectory on its hosting machine that is located under the state directory. The name of the subdirectory is the application ID. The state stores associated with the application are created under this subdirectory. When running multiple instances of the same application on a single machine, this path must be unique for each such instance. + + #### topology.optimization + +> A configuration telling Kafka Streams if it should optimize the topology and what optimizations to apply. Acceptable values are: `StreamsConfig.NO_OPTIMIZATION` (`none`), `StreamsConfig.OPTIMIZE` (`all`) or a comma separated list of specific optimizations: (`StreamsConfig.REUSE_KTABLE_SOURCE_TOPICS` (`reuse.ktable.source.topics`), `StreamsConfig.MERGE_REPARTITION_TOPICS` (`merge.repartition.topics`)). + +We recommend listing specific optimizations in the config for production code so that the structure of your topology will not change unexpectedly during upgrades of the Streams library. + +These optimizations include moving/reducing repartition topics and reusing the source topic as the changelog for source KTables. These optimizations will save on network traffic and storage in Kafka without changing the semantics of your applications. Enabling them is recommended. + +Note that as of 2.3, you need to do two things to enable optimizations. In addition to setting this config to `StreamsConfig.OPTIMIZE`, you'll need to pass in your configuration properties when building your topology by using the overloaded `StreamsBuilder.build(Properties)` method. For example `KafkaStreams myStream = new KafkaStreams(streamsBuilder.build(properties), properties)`. + + #### upgrade.from + +> The version you are upgrading from. It is important to set this config when performing a rolling upgrade to certain versions, as described in the upgrade guide. You should set this config to the appropriate version before bouncing your instances and upgrading them to the newer version. Once everyone is on the newer version, you should remove this config and do a second rolling bounce. It is only necessary to set this config and follow the two-bounce upgrade path when upgrading from below version 2.0, or when upgrading to 2.4+ from any version lower than 2.4. + + ### Kafka consumers, producer and admin client configuration parameters + + You can specify parameters for the Kafka [consumers](/37/javadoc/org/apache/kafka/clients/consumer/package-summary.html), [producers](/37/javadoc/org/apache/kafka/clients/producer/package-summary.html), and [admin client](/37/javadoc/org/apache/kafka/kafka/clients/admin/package-summary.html) that are used internally. The consumer, producer and admin client settings are defined by specifying parameters in a `StreamsConfig` instance. + + In this example, the Kafka [consumer session timeout](/37/javadoc/org/apache/kafka/clients/consumer/ConsumerConfig.html#SESSION_TIMEOUT_MS_CONFIG) is configured to be 60000 milliseconds in the Streams settings: + + + Properties streamsSettings = new Properties(); + // Example of a "normal" setting for Kafka Streams + streamsSettings.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka-broker-01:9092"); + // Customize the Kafka consumer settings of your Streams application + streamsSettings.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 60000); + + #### Naming + + Some consumer, producer and admin client configuration parameters use the same parameter name, and Kafka Streams library itself also uses some parameters that share the same name with its embedded client. For example, `send.buffer.bytes` and `receive.buffer.bytes` are used to configure TCP buffers; `request.timeout.ms` and `retry.backoff.ms` control retries for client request; `retries` are used to configure how many retries are allowed when handling retriable errors from broker request responses. You can avoid duplicate names by prefix parameter names with `consumer.`, `producer.`, or `admin.` (e.g., `consumer.send.buffer.bytes` and `producer.send.buffer.bytes`). + + + Properties streamsSettings = new Properties(); + // same value for consumer, producer, and admin client + streamsSettings.put("PARAMETER_NAME", "value"); + // different values for consumer and producer + streamsSettings.put("consumer.PARAMETER_NAME", "consumer-value"); + streamsSettings.put("producer.PARAMETER_NAME", "producer-value"); + streamsSettings.put("admin.PARAMETER_NAME", "admin-value"); + // alternatively, you can use + streamsSettings.put(StreamsConfig.consumerPrefix("PARAMETER_NAME"), "consumer-value"); + streamsSettings.put(StreamsConfig.producerPrefix("PARAMETER_NAME"), "producer-value"); + streamsSettings.put(StreamsConfig.adminClientPrefix("PARAMETER_NAME"), "admin-value"); + + You could further separate consumer configuration by adding different prefixes: + + * `main.consumer.` for main consumer which is the default consumer of stream source. + * `restore.consumer.` for restore consumer which is in charge of state store recovery. + * `global.consumer.` for global consumer which is used in global KTable construction. + + + + For example, if you only want to set restore consumer config without touching other consumers' settings, you could simply use `restore.consumer.` to set the config. + + + Properties streamsSettings = new Properties(); + // same config value for all consumer types + streamsSettings.put("consumer.PARAMETER_NAME", "general-consumer-value"); + // set a different restore consumer config. This would make restore consumer take restore-consumer-value, + // while main consumer and global consumer stay with general-consumer-value + streamsSettings.put("restore.consumer.PARAMETER_NAME", "restore-consumer-value"); + // alternatively, you can use + streamsSettings.put(StreamsConfig.restoreConsumerPrefix("PARAMETER_NAME"), "restore-consumer-value"); + + Same applied to `main.consumer.` and `main.consumer.`, if you only want to specify one consumer type config. + + Additionally, to configure the internal repartition/changelog topics, you could use the `topic.` prefix, followed by any of the standard topic configs. + + + Properties streamsSettings = new Properties(); + // Override default for both changelog and repartition topics + streamsSettings.put("topic.PARAMETER_NAME", "topic-value"); + // alternatively, you can use + streamsSettings.put(StreamsConfig.topicPrefix("PARAMETER_NAME"), "topic-value"); + + #### Default Values + + Kafka Streams uses different default values for some of the underlying client configs, which are summarized below. For detailed descriptions of these configs, see [Producer Configs](http://kafka.apache.org/0100/documentation.html#producerconfigs) and [Consumer Configs](http://kafka.apache.org/0100/documentation.html#newconsumerconfigs). + + Parameter Name | Corresponding Client | Streams Default + ---|---|--- + auto.offset.reset | Consumer | earliest + linger.ms | Producer | 100 + max.poll.records | Consumer | 1000 + + ### Parameters controlled by Kafka Streams + + Kafka Streams assigns the following configuration parameters. If you try to change `allow.auto.create.topics`, your value is ignored and setting it has no effect in a Kafka Streams application. You can set the other parameters. Kafka Streams sets them to different default values than a plain `KafkaConsumer`. + + Kafka Streams uses the `client.id` parameter to compute derived client IDs for internal clients. If you don't set `client.id`, Kafka Streams sets it to `-`. Parameter Name | Corresponding Client | Streams Default + ---|---|--- + allow.auto.create.topics | Consumer | `false` + auto.offset.reset | Consumer | `earliest` + linger.ms | Producer | `100` + max.poll.records | Consumer | `1000` + + #### enable.auto.commit + +> The consumer auto commit. To guarantee at-least-once processing semantics and turn off auto commits, Kafka Streams overrides this consumer config value to `false`. Consumers will only commit explicitly via _commitSync_ calls when the Kafka Streams library or a user decides to commit the current processing state. + + [Previous](/37/streams/developer-guide/write-streams) [Next](/37/streams/developer-guide/dsl-api) + + * [Documentation](/documentation) + * [Kafka Streams](/streams) + * [Developer Guide](/streams/developer-guide/) + + diff --git a/docs/streams/developer-guide/datatypes.html b/docs/streams/developer-guide/datatypes.html deleted file mode 100644 index 607d133f3b25c..0000000000000 --- a/docs/streams/developer-guide/datatypes.html +++ /dev/null @@ -1,238 +0,0 @@ - - - - - - - - - - - diff --git a/docs/streams/developer-guide/datatypes.md b/docs/streams/developer-guide/datatypes.md new file mode 100644 index 0000000000000..5c40e4d1d7b11 --- /dev/null +++ b/docs/streams/developer-guide/datatypes.md @@ -0,0 +1,127 @@ +--- +title: Data Types and Serialization +description: +weight: 6 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +Every Kafka Streams application must provide Serdes (Serializer/Deserializer) for the data types of record keys and record values (e.g. `java.lang.String`) to materialize the data when necessary. Operations that require such Serdes information include: `stream()`, `table()`, `to()`, `repartition()`, `groupByKey()`, `groupBy()`. + +You can provide Serdes by using either of these methods, but you must use at least one: + + * By setting default Serdes in the `java.util.Properties` config instance. + * By specifying explicit Serdes when calling the appropriate API methods, thus overriding the defaults. + + + + + + +# Configuring Serdes + +Serdes specified in the Streams configuration are used as the default in your Kafka Streams application. Because this config's default is null, you must either set a default Serde by using this configuration or pass in Serdes explicitly, as described below. + + + import org.apache.kafka.common.serialization.Serdes; + import org.apache.kafka.streams.StreamsConfig; + + Properties settings = new Properties(); + // Default serde for keys of data records (here: built-in serde for String type) + settings.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); + // Default serde for values of data records (here: built-in serde for Long type) + settings.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Long().getClass().getName()); + +# Overriding default Serdes + +You can also specify Serdes explicitly by passing them to the appropriate API methods, which overrides the default serde settings: + + + import org.apache.kafka.common.serialization.Serde; + import org.apache.kafka.common.serialization.Serdes; + + final Serde stringSerde = Serdes.String(); + final Serde longSerde = Serdes.Long(); + + // The stream userCountByRegion has type `String` for record keys (for region) + // and type `Long` for record values (for user counts). + KStream userCountByRegion = ...; + userCountByRegion.to("RegionCountsTopic", Produced.with(stringSerde, longSerde)); + +If you want to override serdes selectively, i.e., keep the defaults for some fields, then don't specify the serde whenever you want to leverage the default settings: + + + import org.apache.kafka.common.serialization.Serde; + import org.apache.kafka.common.serialization.Serdes; + + // Use the default serializer for record keys (here: region as String) by not specifying the key serde, + // but override the default serializer for record values (here: userCount as Long). + final Serde longSerde = Serdes.Long(); + KStream userCountByRegion = ...; + userCountByRegion.to("RegionCountsTopic", Produced.valueSerde(Serdes.Long())); + +If some of your incoming records are corrupted or ill-formatted, they will cause the deserializer class to report an error. Since 1.0.x we have introduced an `DeserializationExceptionHandler` interface which allows you to customize how to handle such records. The customized implementation of the interface can be specified via the `StreamsConfig`. For more details, please feel free to read the [Configuring a Streams Application](config-streams.html#default-deserialization-exception-handler) section. + +# Available Serdes + +## Primitive and basic types + +Apache Kafka includes several built-in serde implementations for Java primitives and basic types such as `byte[]` in its `kafka-clients` Maven artifact: + + + + org.apache.kafka + kafka-clients + 3.7.2 + + +This artifact provides the following serde implementations under the package [org.apache.kafka.common.serialization](https://github.com/apache/kafka/blob/3.7/clients/src/main/java/org/apache/kafka/common/serialization), which you can leverage when e.g., defining default serializers in your Streams configuration. + +Data type | Serde +---|--- +byte[] | `Serdes.ByteArray()`, `Serdes.Bytes()` (see tip below) +ByteBuffer | `Serdes.ByteBuffer()` +Double | `Serdes.Double()` +Integer | `Serdes.Integer()` +Long | `Serdes.Long()` +String | `Serdes.String()` +UUID | `Serdes.UUID()` +Void | `Serdes.Void()` +List | `Serdes.ListSerde()` +Boolean | `Serdes.Boolean()` + +**Tip** + +[Bytes](https://github.com/apache/kafka/blob/3.7/clients/src/main/java/org/apache/kafka/common/utils/Bytes.java) is a wrapper for Java's `byte[]` (byte array) that supports proper equality and ordering semantics. You may want to consider using `Bytes` instead of `byte[]` in your applications. + +## JSON + +The Kafka Streams code examples also include a basic serde implementation for JSON: + + * [PageViewTypedDemo](https://github.com/apache/kafka/blob/3.7/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java#L83) + + + +As shown in the example, you can use JSONSerdes inner classes `Serdes.serdeFrom(, )` to construct JSON compatible serializers and deserializers. + +# Implementing custom Serdes + +If you need to implement custom Serdes, your best starting point is to take a look at the source code references of existing Serdes (see previous section). Typically, your workflow will be similar to: + + 1. Write a _serializer_ for your data type `T` by implementing [org.apache.kafka.common.serialization.Serializer](https://github.com/apache/kafka/blob/3.7/clients/src/main/java/org/apache/kafka/common/serialization/Serializer.java). + 2. Write a _deserializer_ for `T` by implementing [org.apache.kafka.common.serialization.Deserializer](https://github.com/apache/kafka/blob/3.7/clients/src/main/java/org/apache/kafka/common/serialization/Deserializer.java). + 3. Write a _serde_ for `T` by implementing [org.apache.kafka.common.serialization.Serde](https://github.com/apache/kafka/blob/3.7/clients/src/main/java/org/apache/kafka/common/serialization/Serde.java), which you either do manually (see existing Serdes in the previous section) or by leveraging helper functions in [Serdes](https://github.com/apache/kafka/blob/3.7/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java) such as `Serdes.serdeFrom(Serializer, Deserializer)`. Note that you will need to implement your own class (that has no generic types) if you want to use your custom serde in the configuration provided to `KafkaStreams`. If your serde class has generic types or you use `Serdes.serdeFrom(Serializer, Deserializer)`, you can pass your serde only via methods calls (for example `builder.stream("topicName", Consumed.with(...))`). + + + +# Kafka Streams DSL for Scala Implicit Serdes[](scala-dsl-serdes "Permalink to this headline") + +When using the [Kafka Streams DSL for Scala](dsl-api.html#scala-dsl) you're not required to configure a default Serdes. In fact, it's not supported. Serdes are instead provided implicitly by default implementations for common primitive datatypes. See the [Implicit Serdes](dsl-api.html#scala-dsl-implicit-serdes) and [User-Defined Serdes](dsl-api.html#scala-dsl-user-defined-serdes) sections in the DSL API documentation for details + + * [Documentation](/documentation) + * [Kafka Streams](/streams) + * [Developer Guide](/streams/developer-guide/) + + diff --git a/docs/streams/developer-guide/dsl-api.html b/docs/streams/developer-guide/dsl-api.html deleted file mode 100644 index fd5c22cae33bb..0000000000000 --- a/docs/streams/developer-guide/dsl-api.html +++ /dev/null @@ -1,3974 +0,0 @@ - - - - - - - - - - - diff --git a/docs/streams/developer-guide/dsl-api.md b/docs/streams/developer-guide/dsl-api.md new file mode 100644 index 0000000000000..c24faa4d506de --- /dev/null +++ b/docs/streams/developer-guide/dsl-api.md @@ -0,0 +1,2375 @@ +--- +title: Streams DSL +description: +weight: 3 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +The Kafka Streams DSL (Domain Specific Language) is built on top of the Streams Processor API. It is the recommended for most users, especially beginners. Most data processing operations can be expressed in just a few lines of DSL code. + + * Aggregating + * Joining + * Join co-partitioning requirements + * KStream-KStream Join + * KTable-KTable Equi-Join + * KTable-KTable Foreign-Key Join + * KStream-KTable Join + * KStream-GlobalKTable Join + * Windowing + * Hopping time windows + * Tumbling time windows + * Sliding time windows + * Session Windows + * Window Final Results + * Applying processors and transformers (Processor API integration) + * Naming Operators in a Streams DSL application + * Controlling KTable update rate + * Using timestamp-based semantics for table processors + * Writing streams back to Kafka + * Testing a Streams application + * Kafka Streams DSL for Scala + * Sample Usage + * Implicit Serdes + * User-Defined Serdes + + + +# Overview + +In comparison to the [Processor API](processor-api.html#streams-developer-guide-processor-api), only the DSL supports: + + * Built-in abstractions for [streams and tables](../core-concepts.html#streams_concepts_duality) in the form of KStream, KTable, and GlobalKTable. Having first-class support for streams and tables is crucial because, in practice, most use cases require not just either streams or databases/tables, but a combination of both. For example, if your use case is to create a customer 360-degree view that is updated in real-time, what your application will be doing is transforming many input _streams_ of customer-related events into an output _table_ that contains a continuously updated 360-degree view of your customers. + * Declarative, functional programming style with stateless transformations (e.g. `map` and `filter`) as well as stateful transformations such as aggregations (e.g. `count` and `reduce`), joins (e.g. `leftJoin`), and windowing (e.g. session windows). + + + +With the DSL, you can define [processor topologies](../core-concepts.html#streams_topology) (i.e., the logical processing plan) in your application. The steps to accomplish this are: + + 1. Specify one or more input streams that are read from Kafka topics. + 2. Compose transformations on these streams. + 3. Write the resulting output streams back to Kafka topics, or expose the processing results of your application directly to other applications through [interactive queries](interactive-queries.html#streams-developer-guide-interactive-queries) (e.g., via a REST API). + + + +After the application is run, the defined processor topologies are continuously executed (i.e., the processing plan is put into action). A step-by-step guide for writing a stream processing application using the DSL is provided below. + +For a complete list of available API functionality, see also the [Streams](/37/javadoc/org/apache/kafka/streams/package-summary.html) API docs. + +### KStream + +Only the **Kafka Streams DSL** has the notion of a `KStream`. + +A **KStream** is an abstraction of a **record stream** , where each data record represents a self-contained datum in the unbounded data set. Using the table analogy, data records in a record stream are always interpreted as an "INSERT" \-- think: adding more entries to an append-only ledger -- because no record replaces an existing row with the same key. Examples are a credit card transaction, a page view event, or a server log entry. + +To illustrate, let's imagine the following two data records are being sent to the stream: + +("alice", 1) --> ("alice", 3) + +If your stream processing application were to sum the values per user, it would return `4` for `alice`. Why? Because the second data record would not be considered an update of the previous record. Compare this behavior of KStream to `KTable` below, which would return `3` for `alice`. + +### KTable + +Only the **Kafka Streams DSL** has the notion of a `KTable`. + +A **KTable** is an abstraction of a **changelog stream** , where each data record represents an update. More precisely, the value in a data record is interpreted as an "UPDATE" of the last value for the same record key, if any (if a corresponding key doesn't exist yet, the update will be considered an INSERT). Using the table analogy, a data record in a changelog stream is interpreted as an UPSERT aka INSERT/UPDATE because any existing row with the same key is overwritten. Also, `null` values are interpreted in a special way: a record with a `null` value represents a "DELETE" or tombstone for the record's key. + +To illustrate, let's imagine the following two data records are being sent to the stream: + +("alice", 1) --> ("alice", 3) + +If your stream processing application were to sum the values per user, it would return `3` for `alice`. Why? Because the second data record would be considered an update of the previous record. + +**Effects of Kafka's log compaction:** Another way of thinking about KStream and KTable is as follows: If you were to store a KTable into a Kafka topic, you'd probably want to enable Kafka's [log compaction](http://kafka.apache.org/documentation.html#compaction) feature, e.g. to save storage space. + +However, it would not be safe to enable log compaction in the case of a KStream because, as soon as log compaction would begin purging older data records of the same key, it would break the semantics of the data. To pick up the illustration example again, you'd suddenly get a `3` for `alice` instead of a `4` because log compaction would have removed the `("alice", 1)` data record. Hence log compaction is perfectly safe for a KTable (changelog stream) but it is a mistake for a KStream (record stream). + +We have already seen an example of a changelog stream in the section [streams and tables](../core-concepts.html#streams_concepts_duality). Another example are change data capture (CDC) records in the changelog of a relational database, representing which row in a database table was inserted, updated, or deleted. + +KTable also provides an ability to look up _current_ values of data records by keys. This table-lookup functionality is available through **join operations** (see also **Joining** in the Developer Guide) as well as through **Interactive Queries**. + +### GlobalKTable + +Only the **Kafka Streams DSL** has the notion of a **GlobalKTable**. + +Like a **KTable** , a **GlobalKTable** is an abstraction of a **changelog stream** , where each data record represents an update. + +A GlobalKTable differs from a KTable in the data that they are being populated with, i.e. which data from the underlying Kafka topic is being read into the respective table. Slightly simplified, imagine you have an input topic with 5 partitions. In your application, you want to read this topic into a table. Also, you want to run your application across 5 application instances for **maximum parallelism**. + + * If you read the input topic into a **KTable** , then the "local" KTable instance of each application instance will be populated with data **from only 1 partition** of the topic's 5 partitions. + * If you read the input topic into a **GlobalKTable** , then the local GlobalKTable instance of each application instance will be populated with data **from all partitions of the topic**. + + + +GlobalKTable provides the ability to look up _current_ values of data records by keys. This table-lookup functionality is available through `join operations`. Note that a GlobalKTable has **no** notion of time in contrast to a KTable. + +Benefits of global tables: + + * More convenient and/or efficient **joins** : Notably, global tables allow you to perform star joins, they support "foreign-key" lookups (i.e., you can lookup data in the table not just by record key, but also by data in the record values), and they are more efficient when chaining multiple joins. Also, when joining against a global table, the input data does not need to be **co-partitioned**. + * Can be used to "broadcast" information to all the running instances of your application. + + + +Downsides of global tables: + + * Increased local storage consumption compared to the (partitioned) KTable because the entire topic is tracked. + * Increased network and Kafka broker load compared to the (partitioned) KTable because the entire topic is read. + + + +# Creating source streams from Kafka + +You can easily read data from Kafka topics into your application. The following operations are supported. + +Reading from Kafka | Description +---|--- +**Stream** + + * _input topics_ -> KStream + +| Creates a KStream from the specified Kafka input topics and interprets the data as a record stream. A `KStream` represents a _partitioned_ record stream. [(details)](/37/javadoc/org/apache/kafka/streams/StreamsBuilder.html#stream\(java.lang.String\)) In the case of a KStream, the local KStream instance of every application instance will be populated with data from only **a subset** of the partitions of the input topic. Collectively, across all application instances, all input topic partitions are read and processed. + + + import org.apache.kafka.common.serialization.Serdes; + import org.apache.kafka.streams.StreamsBuilder; + import org.apache.kafka.streams.kstream.KStream; + + StreamsBuilder builder = new StreamsBuilder(); + + KStream wordCounts = builder.stream( + "word-counts-input-topic", /* input topic */ + Consumed.with( + Serdes.String(), /* key serde */ + Serdes.Long() /* value serde */ + ); + +If you do not specify Serdes explicitly, the default Serdes from the [configuration](config-streams.html#streams-developer-guide-configuration) are used. You **must specify Serdes explicitly** if the key or value types of the records in the Kafka input topics do not match the configured default Serdes. For information about configuring default Serdes, available Serdes, and implementing your own custom Serdes see [Data Types and Serialization](datatypes.html#streams-developer-guide-serdes). Several variants of `stream` exist. For example, you can specify a regex pattern for input topics to read from (note that all matching topics will be part of the same input topic group, and the work will not be parallelized for different topics if subscribed to in this way). +**Table** + + * _input topic_ -> KTable + +| Reads the specified Kafka input topic into a KTable. The topic is interpreted as a changelog stream, where records with the same key are interpreted as UPSERT aka INSERT/UPDATE (when the record value is not `null`) or as DELETE (when the value is `null`) for that key. [(details)](/37/javadoc/org/apache/kafka/streams/StreamsBuilder.html#table-java.lang.String\(java.lang.String\)) In the case of a KTable, the local KTable instance of every application instance will be populated with data from only **a subset** of the partitions of the input topic. Collectively, across all application instances, all input topic partitions are read and processed. You must provide a name for the table (more precisely, for the internal [state store](../architecture.html#streams_architecture_state) that backs the table). This is required for supporting [interactive queries](interactive-queries.html#streams-developer-guide-interactive-queries) against the table. When a name is not provided the table will not be queryable and an internal name will be provided for the state store. If you do not specify Serdes explicitly, the default Serdes from the [configuration](config-streams.html#streams-developer-guide-configuration) are used. You **must specify Serdes explicitly** if the key or value types of the records in the Kafka input topics do not match the configured default Serdes. For information about configuring default Serdes, available Serdes, and implementing your own custom Serdes see [Data Types and Serialization](datatypes.html#streams-developer-guide-serdes). Several variants of `table` exist, for example to specify the `auto.offset.reset` policy to be used when reading from the input topic. +**Global Table** + + * _input topic_ -> GlobalKTable + +| Reads the specified Kafka input topic into a GlobalKTable. The topic is interpreted as a changelog stream, where records with the same key are interpreted as UPSERT aka INSERT/UPDATE (when the record value is not `null`) or as DELETE (when the value is `null`) for that key. [(details)](/37/javadoc/org/apache/kafka/streams/StreamsBuilder.html#globalTable-java.lang.String\(java.lang.String\)) In the case of a GlobalKTable, the local GlobalKTable instance of every application instance will be populated with data from **all** the partitions of the input topic. You must provide a name for the table (more precisely, for the internal [state store](../architecture.html#streams_architecture_state) that backs the table). This is required for supporting [interactive queries](interactive-queries.html#streams-developer-guide-interactive-queries) against the table. When a name is not provided the table will not be queryable and an internal name will be provided for the state store. + + + import org.apache.kafka.common.serialization.Serdes; + import org.apache.kafka.streams.StreamsBuilder; + import org.apache.kafka.streams.kstream.GlobalKTable; + + StreamsBuilder builder = new StreamsBuilder(); + + GlobalKTable wordCounts = builder.globalTable( + "word-counts-input-topic", + Materialized.>as( + "word-counts-global-store" /* table/store name */) + .withKeySerde(Serdes.String()) /* key serde */ + .withValueSerde(Serdes.Long()) /* value serde */ + ); + +You **must specify Serdes explicitly** if the key or value types of the records in the Kafka input topics do not match the configured default Serdes. For information about configuring default Serdes, available Serdes, and implementing your own custom Serdes see [Data Types and Serialization](datatypes.html#streams-developer-guide-serdes). Several variants of `globalTable` exist to e.g. specify explicit Serdes. + +# Transform a stream + +The KStream and KTable interfaces support a variety of transformation operations. Each of these operations can be translated into one or more connected processors into the underlying processor topology. Since KStream and KTable are strongly typed, all of these transformation operations are defined as generic functions where users could specify the input and output data types. + +Some KStream transformations may generate one or more KStream objects, for example: \- `filter` and `map` on a KStream will generate another KStream \- `split` on KStream can generate multiple KStreams + +Some others may generate a KTable object, for example an aggregation of a KStream also yields a KTable. This allows Kafka Streams to continuously update the computed value upon arrivals of [out-of-order records](../core-concepts.html#streams_concepts_aggregations) after it has already been produced to the downstream transformation operators. + +All KTable transformation operations can only generate another KTable. However, the Kafka Streams DSL does provide a special function that converts a KTable representation into a KStream. All of these transformation methods can be chained together to compose a complex processor topology. + +These transformation operations are described in the following subsections: + + * Stateless transformations + * Stateful transformations + + + +## Stateless transformations + +Stateless transformations do not require state for processing and they do not require a state store associated with the stream processor. Kafka 0.11.0 and later allows you to materialize the result from a stateless `KTable` transformation. This allows the result to be queried through [interactive queries](interactive-queries.html#streams-developer-guide-interactive-queries). To materialize a `KTable`, each of the below stateless operations [can be augmented](interactive-queries.html#streams-developer-guide-interactive-queries-local-key-value-stores) with an optional `queryableStoreName` argument. + +Transformation | Description +---|--- +**Branch** + + * KStream -> BranchedKStream + +| Branch (or split) a `KStream` based on the supplied predicates into one or more `KStream` instances. ([details](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#split\(\))) Predicates are evaluated in order. A record is placed to one and only one output stream on the first match: if the n-th predicate evaluates to true, the record is placed to n-th stream. If a record does not match any predicates, it will be routed to the default branch, or dropped if no default branch is created. Branching is useful, for example, to route records to different downstream topics. + + + KStream stream = ...; + Map> branches = + stream.split(Named.as("Branch-")) + .branch((key, value) -> key.startsWith("A"), /* first predicate */ + Branched.as("A")) + .branch((key, value) -> key.startsWith("B"), /* second predicate */ + Branched.as("B")) + .defaultBranch(Branched.as("C")) /* default branch */ + ); + + // KStream branches.get("Branch-A") contains all records whose keys start with "A" + // KStream branches.get("Branch-B") contains all records whose keys start with "B" + // KStream branches.get("Branch-C") contains all other records + + // Java 7 example: cf. `filter` for how to create `Predicate` instances + +**Filter** + + * KStream -> KStream + * KTable -> KTable + +| Evaluates a boolean function for each element and retains those for which the function returns true. ([KStream details](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#filter-org.apache.kafka.streams.kstream.Predicate-), [KTable details](/37/javadoc/org/apache/kafka/streams/kstream/KTable.html#filter-org.apache.kafka.streams.kstream.Predicate-)) + + + KStream stream = ...; + + // A filter that selects (keeps) only positive numbers + // Java 8+ example, using lambda expressions + KStream onlyPositives = stream.filter((key, value) -> value > 0); + + // Java 7 example + KStream onlyPositives = stream.filter( + new Predicate() { + @Override + public boolean test(String key, Long value) { + return value > 0; + } + }); + +**Inverse Filter** + + * KStream -> KStream + * KTable -> KTable + +| Evaluates a boolean function for each element and drops those for which the function returns true. ([KStream details](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#filterNot-org.apache.kafka.streams.kstream.Predicate-), [KTable details](/37/javadoc/org/apache/kafka/streams/kstream/KTable.html#filterNot-org.apache.kafka.streams.kstream.Predicate-)) + + + KStream stream = ...; + + // An inverse filter that discards any negative numbers or zero + // Java 8+ example, using lambda expressions + KStream onlyPositives = stream.filterNot((key, value) -> value <= 0); + + // Java 7 example + KStream onlyPositives = stream.filterNot( + new Predicate() { + @Override + public boolean test(String key, Long value) { + return value <= 0; + } + }); + +**FlatMap** + + * KStream -> KStream + +| Takes one record and produces zero, one, or more records. You can modify the record keys and values, including their types. ([details](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#flatMap-org.apache.kafka.streams.kstream.KeyValueMapper-)) **Marks the stream for data re-partitioning:** Applying a grouping or a join after `flatMap` will result in re-partitioning of the records. If possible use `flatMapValues` instead, which will not cause data re-partitioning. + + + KStream stream = ...; + KStream transformed = stream.flatMap( + // Here, we generate two output records for each input record. + // We also change the key and value types. + // Example: (345L, "Hello") -> ("HELLO", 1000), ("hello", 9000) + (key, value) -> { + List> result = new LinkedList<>(); + result.add(KeyValue.pair(value.toUpperCase(), 1000)); + result.add(KeyValue.pair(value.toLowerCase(), 9000)); + return result; + } + ); + + // Java 7 example: cf. `map` for how to create `KeyValueMapper` instances + +**FlatMapValues** + + * KStream -> KStream + +| Takes one record and produces zero, one, or more records, while retaining the key of the original record. You can modify the record values and the value type. ([details](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#flatMapValues-org.apache.kafka.streams.kstream.ValueMapper-)) `flatMapValues` is preferable to `flatMap` because it will not cause data re-partitioning. However, you cannot modify the key or key type like `flatMap` does. + + + // Split a sentence into words. + KStream sentences = ...; + KStream words = sentences.flatMapValues(value -> Arrays.asList(value.split("\s+"))); + + // Java 7 example: cf. `mapValues` for how to create `ValueMapper` instances + +**Foreach** + + * KStream -> void + * KStream -> void + * KTable -> void + +| **Terminal operation.** Performs a stateless action on each record. ([details](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#foreach-org.apache.kafka.streams.kstream.ForeachAction-)) You would use `foreach` to cause _side effects_ based on the input data (similar to `peek`) and then _stop_ _further processing_ of the input data (unlike `peek`, which is not a terminal operation). **Note on processing guarantees:** Any side effects of an action (such as writing to external systems) are not trackable by Kafka, which means they will typically not benefit from Kafka's processing guarantees. + + + KStream stream = ...; + + // Print the contents of the KStream to the local console. + // Java 8+ example, using lambda expressions + stream.foreach((key, value) -> System.out.println(key + " => " + value)); + + // Java 7 example + stream.foreach( + new ForeachAction() { + @Override + public void apply(String key, Long value) { + System.out.println(key + " => " + value); + } + }); + +**GroupByKey** + + * KStream -> KGroupedStream + +| Groups the records by the existing key. ([details](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#groupByKey--)) Grouping is a prerequisite for aggregating a stream or a table and ensures that data is properly partitioned ("keyed") for subsequent operations. **When to set explicit Serdes:** Variants of `groupByKey` exist to override the configured default Serdes of your application, which **you** **must do** if the key and/or value types of the resulting `KGroupedStream` do not match the configured default Serdes. **Note** **Grouping vs. Windowing:** A related operation is windowing, which lets you control how to "sub-group" the grouped records _of the same key_ into so-called _windows_ for stateful operations such as windowed aggregations or windowed joins. **Causes data re-partitioning if and only if the stream was marked for re-partitioning.** `groupByKey` is preferable to `groupBy` because it re-partitions data only if the stream was already marked for re-partitioning. However, `groupByKey` does not allow you to modify the key or key type like `groupBy` does. + + + KStream stream = ...; + + // Group by the existing key, using the application's configured + // default serdes for keys and values. + KGroupedStream groupedStream = stream.groupByKey(); + + // When the key and/or value types do not match the configured + // default serdes, we must explicitly specify serdes. + KGroupedStream groupedStream = stream.groupByKey( + Grouped.with( + Serdes.ByteArray(), /* key */ + Serdes.String()) /* value */ + ); + +**GroupBy** + + * KStream -> KGroupedStream + * KTable -> KGroupedTable + +| Groups the records by a _new_ key, which may be of a different key type. When grouping a table, you may also specify a new value and value type. `groupBy` is a shorthand for `selectKey(...).groupByKey()`. ([KStream details](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#groupBy-org.apache.kafka.streams.kstream.KeyValueMapper-), [KTable details](/37/javadoc/org/apache/kafka/streams/kstream/KTable.html#groupBy-org.apache.kafka.streams.kstream.KeyValueMapper-)) Grouping is a prerequisite for aggregating a stream or a table and ensures that data is properly partitioned ("keyed") for subsequent operations. **When to set explicit Serdes:** Variants of `groupBy` exist to override the configured default Serdes of your application, which **you must** **do** if the key and/or value types of the resulting `KGroupedStream` or `KGroupedTable` do not match the configured default Serdes. **Note** **Grouping vs. Windowing:** A related operation is windowing, which lets you control how to "sub-group" the grouped records _of the same key_ into so-called _windows_ for stateful operations such as windowed aggregations or windowed joins. **Always causes data re-partitioning:** `groupBy` always causes data re-partitioning. If possible use `groupByKey` instead, which will re-partition data only if required. + + + KStream stream = ...; + KTable table = ...; + + // Java 8+ examples, using lambda expressions + + // Group the stream by a new key and key type + KGroupedStream groupedStream = stream.groupBy( + (key, value) -> value, + Grouped.with( + Serdes.String(), /* key (note: type was modified) */ + Serdes.String()) /* value */ + ); + + // Group the table by a new key and key type, and also modify the value and value type. + KGroupedTable groupedTable = table.groupBy( + (key, value) -> KeyValue.pair(value, value.length()), + Grouped.with( + Serdes.String(), /* key (note: type was modified) */ + Serdes.Integer()) /* value (note: type was modified) */ + ); + + + // Java 7 examples + + // Group the stream by a new key and key type + KGroupedStream groupedStream = stream.groupBy( + new KeyValueMapper>() { + @Override + public String apply(byte[] key, String value) { + return value; + } + }, + Grouped.with( + Serdes.String(), /* key (note: type was modified) */ + Serdes.String()) /* value */ + ); + + // Group the table by a new key and key type, and also modify the value and value type. + KGroupedTable groupedTable = table.groupBy( + new KeyValueMapper>() { + @Override + public KeyValue apply(byte[] key, String value) { + return KeyValue.pair(value, value.length()); + } + }, + Grouped.with( + Serdes.String(), /* key (note: type was modified) */ + Serdes.Integer()) /* value (note: type was modified) */ + ); + +**Cogroup** + + * KGroupedStream -> CogroupedKStream + * CogroupedKStream -> CogroupedKStream + +| Cogrouping allows to aggregate multiple input streams in a single operation. The different (already grouped) input streams must have the same key type and may have different values types. [KGroupedStream#cogroup()](/37/javadoc/org/apache/kafka/streams/kstream/KGroupedStream.html#cogroup) creates a new cogrouped stream with a single input stream, while [CogroupedKStream#cogroup()](/37/javadoc/org/apache/kafka/streams/kstream/CogroupedKStream.html#cogroup) adds a grouped stream to an existing cogrouped stream. A `CogroupedKStream` may be [windowed](/37/javadoc/org/apache/kafka/streams/kstream/CogroupedKStream.html#windowedBy) before it is [aggregated](/37/javadoc/org/apache/kafka/streams/kstream/CogroupedKStream.html#aggregate). Cogroup does not cause a repartition as it has the prerequisite that the input streams are grouped. In the process of creating these groups they will have already been repartitioned if the stream was already marked for repartitioning. + + + KStream stream = ...; + KStream stream2 = ...; + + // Group by the existing key, using the application's configured + // default serdes for keys and values. + KGroupedStream groupedStream = stream.groupByKey(); + KGroupedStream groupedStream2 = stream2.groupByKey(); + CogroupedKStream cogroupedStream = groupedStream.cogroup(aggregator1).cogroup(groupedStream2, aggregator2); + + KTable table = cogroupedStream.aggregate(initializer); + + KTable table2 = cogroupedStream.windowedBy(TimeWindows.ofSizeWithNoGrace(Duration.ofMillis(500))).aggregate(initializer); + +**Map** + + * KStream -> KStream + +| Takes one record and produces one record. You can modify the record key and value, including their types. ([details](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#map-org.apache.kafka.streams.kstream.KeyValueMapper-)) **Marks the stream for data re-partitioning:** Applying a grouping or a join after `map` will result in re-partitioning of the records. If possible use `mapValues` instead, which will not cause data re-partitioning. + + + KStream stream = ...; + + // Java 8+ example, using lambda expressions + // Note how we change the key and the key type (similar to `selectKey`) + // as well as the value and the value type. + KStream transformed = stream.map( + (key, value) -> KeyValue.pair(value.toLowerCase(), value.length())); + + // Java 7 example + KStream transformed = stream.map( + new KeyValueMapper>() { + @Override + public KeyValue apply(byte[] key, String value) { + return new KeyValue<>(value.toLowerCase(), value.length()); + } + }); + +**Map (values only)** + + * KStream -> KStream + * KTable -> KTable + +| Takes one record and produces one record, while retaining the key of the original record. You can modify the record value and the value type. ([KStream details](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#mapValues-org.apache.kafka.streams.kstream.ValueMapper-), [KTable details](/37/javadoc/org/apache/kafka/streams/kstream/KTable.html#mapValues-org.apache.kafka.streams.kstream.ValueMapper-)) `mapValues` is preferable to `map` because it will not cause data re-partitioning. However, it does not allow you to modify the key or key type like `map` does. + + + KStream stream = ...; + + // Java 8+ example, using lambda expressions + KStream uppercased = stream.mapValues(value -> value.toUpperCase()); + + // Java 7 example + KStream uppercased = stream.mapValues( + new ValueMapper() { + @Override + public String apply(String s) { + return s.toUpperCase(); + } + }); + +**Merge** + + * KStream -> KStream + +| Merges records of two streams into one larger stream. ([details](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#merge-org.apache.kafka.streams.kstream.KStream-)) There is no ordering guarantee between records from different streams in the merged stream. Relative order is preserved within each input stream though (ie, records within the same input stream are processed in order) + + + KStream stream1 = ...; + + KStream stream2 = ...; + + KStream merged = stream1.merge(stream2); + +**Peek** + + * KStream -> KStream + +| Performs a stateless action on each record, and returns an unchanged stream. ([details](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#peek-org.apache.kafka.streams.kstream.ForeachAction-)) You would use `peek` to cause _side effects_ based on the input data (similar to `foreach`) and _continue_ _processing_ the input data (unlike `foreach`, which is a terminal operation). `peek` returns the input stream as-is; if you need to modify the input stream, use `map` or `mapValues` instead. `peek` is helpful for use cases such as logging or tracking metrics or for debugging and troubleshooting. **Note on processing guarantees:** Any side effects of an action (such as writing to external systems) are not trackable by Kafka, which means they will typically not benefit from Kafka's processing guarantees. + + + KStream stream = ...; + + // Java 8+ example, using lambda expressions + KStream unmodifiedStream = stream.peek( + (key, value) -> System.out.println("key=" + key + ", value=" + value)); + + // Java 7 example + KStream unmodifiedStream = stream.peek( + new ForeachAction() { + @Override + public void apply(byte[] key, String value) { + System.out.println("key=" + key + ", value=" + value); + } + }); + +**Print** + + * KStream -> void + +| **Terminal operation.** Prints the records to `System.out`. See Javadocs for serde and `toString()` caveats. ([details](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#print--)) Calling `print()` is the same as calling `foreach((key, value) -> System.out.println(key + ", " + value))` `print` is mainly for debugging/testing purposes, and it will try to flush on each record print. Hence it **should not** be used for production usage if performance requirements are concerned. + + + KStream stream = ...; + // print to sysout + stream.print(); + + // print to file with a custom label + stream.print(Printed.toFile("streams.out").withLabel("streams")); + +**SelectKey** + + * KStream -> KStream + +| Assigns a new key - possibly of a new key type - to each record. ([details](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#selectKey-org.apache.kafka.streams.kstream.KeyValueMapper-)) Calling `selectKey(mapper)` is the same as calling `map((key, value) -> mapper(key, value), value)`. **Marks the stream for data re-partitioning:** Applying a grouping or a join after `selectKey` will result in re-partitioning of the records. + + + KStream stream = ...; + + // Derive a new record key from the record's value. Note how the key type changes, too. + // Java 8+ example, using lambda expressions + KStream rekeyed = stream.selectKey((key, value) -> value.split(" ")[0]) + + // Java 7 example + KStream rekeyed = stream.selectKey( + new KeyValueMapper() { + @Override + public String apply(byte[] key, String value) { + return value.split(" ")[0]; + } + }); + +**Table to Stream** + + * KTable -> KStream + +| Get the changelog stream of this table. ([details](/37/javadoc/org/apache/kafka/streams/kstream/KTable.html#toStream--)) + + + KTable table = ...; + + // Also, a variant of `toStream` exists that allows you + // to select a new key for the resulting stream. + KStream stream = table.toStream(); + +**Stream to Table** + + * KStream -> KTable + +| Convert an event stream into a table, or say a changelog stream. ([details](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#toTable--)) + + + KStream stream = ...; + + KTable table = stream.toTable(); + +**Repartition** + + * KStream -> KStream + +| Manually trigger repartitioning of the stream with desired number of partitions. ([details](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#repartition--)) `repartition()` is similar to `through()` however Kafka Streams will manage the topic for you. Generated topic is treated as internal topic, as a result data will be purged automatically as any other internal repartition topic. In addition, you can specify the desired number of partitions, which allows to easily scale in/out downstream sub-topologies. `repartition()` operation always triggers repartitioning of the stream, as a result it can be used with embedded Processor API methods (like `transform()` et al.) that do not trigger auto repartitioning when key changing operation is performed beforehand. + + + KStream stream = ... ; + KStream repartitionedStream = stream.repartition(Repartitioned.numberOfPartitions(10)); + +## Stateful transformations + +Stateful transformations depend on state for processing inputs and producing outputs and require a [state store](../architecture.html#streams_architecture_state) associated with the stream processor. For example, in aggregating operations, a windowing state store is used to collect the latest aggregation results per window. In join operations, a windowing state store is used to collect all of the records received so far within the defined window boundary. + +**Note:** Following store types are used regardless of the possibly specified type (via the parameter `materialized`): + + * non-windowed aggregations and non-windowed KTables use [TimestampedKeyValueStore](/37/javadoc/org/apache/kafka/streams/state/TimestampedKeyValueStore.html)s or [VersionedKeyValueStore](/37/javadoc/org/apache/kafka/streams/state/VersionedKeyValueStore.html)s, depending on whether the parameter `materialized` is versioned + * time-windowed aggregations and KStream-KStream joins use [TimestampedWindowStore](/37/javadoc/org/apache/kafka/streams/state/TimestampedWindowStore.html)s + * session windowed aggregations use [SessionStore](/37/javadoc/org/apache/kafka/streams/state/SessionStore.html)s (there is no timestamped session store as of now) + + + +Note, that state stores are fault-tolerant. In case of failure, Kafka Streams guarantees to fully restore all state stores prior to resuming the processing. See [Fault Tolerance](../architecture.html#streams_architecture_recovery) for further information. + +Available stateful transformations in the DSL include: + + * Aggregating + * Joining + * Windowing (as part of aggregations and joins) + * Applying custom processors and transformers, which may be stateful, for Processor API integration + + + +The following diagram shows their relationships: + +![](/37/images/streams-stateful_operations.png) + +Stateful transformations in the DSL. + +Here is an example of a stateful application: the WordCount algorithm. + +WordCount example in Java 8+, using lambda expressions: + + + // Assume the record values represent lines of text. For the sake of this example, you can ignore + // whatever may be stored in the record keys. + KStream textLines = ...; + + KStream wordCounts = textLines + // Split each text line, by whitespace, into words. The text lines are the record + // values, i.e. you can ignore whatever data is in the record keys and thus invoke + // `flatMapValues` instead of the more generic `flatMap`. + .flatMapValues(value -> Arrays.asList(value.toLowerCase().split("\W+"))) + // Group the stream by word to ensure the key of the record is the word. + .groupBy((key, word) -> word) + // Count the occurrences of each word (record key). + // + // This will change the stream type from `KGroupedStream` to + // `KTable` (word -> count). + .count() + // Convert the `KTable` into a `KStream`. + .toStream(); + +WordCount example in Java 7: + + + // Code below is equivalent to the previous Java 8+ example above. + KStream textLines = ...; + + KStream wordCounts = textLines + .flatMapValues(new ValueMapper>() { + @Override + public Iterable apply(String value) { + return Arrays.asList(value.toLowerCase().split("\W+")); + } + }) + .groupBy(new KeyValueMapper>() { + @Override + public String apply(String key, String word) { + return word; + } + }) + .count() + .toStream(); + +### Aggregating + +After records are grouped by key via `groupByKey` or `groupBy` - and thus represented as either a `KGroupedStream` or a `KGroupedTable`, they can be aggregated via an operation such as `reduce`. Aggregations are key-based operations, which means that they always operate over records (notably record values) of the same key. You can perform aggregations on windowed or non-windowed data. + +Transformation | Description +---|--- +**Aggregate** + + * KGroupedStream -> KTable + * KGroupedTable -> KTable + +| **Rolling aggregation.** Aggregates the values of (non-windowed) records by the grouped key or cogrouped. Aggregating is a generalization of `reduce` and allows, for example, the aggregate value to have a different type than the input values. ([KGroupedStream details](/37/javadoc/org/apache/kafka/streams/kstream/KGroupedStream.html), [KGroupedTable details](/37/javadoc/org/apache/kafka/streams/kstream/KGroupedTable.html) [KGroupedTable details](/37/javadoc/org/apache/kafka/streams/kstream/CogroupedKStream.html)) When aggregating a _grouped stream_ , you must provide an initializer (e.g., `aggValue = 0`) and an "adder" aggregator (e.g., `aggValue + curValue`). When aggregating a _grouped table_ , you must additionally provide a "subtractor" aggregator (think: `aggValue - oldValue`). When aggregating a _cogrouped stream_ , the actual aggregators are provided for each input stream in the prior `cogroup()`calls, and thus you only need to provide an initializer (e.g., `aggValue = 0`) Several variants of `aggregate` exist, see Javadocs for details. + + + KGroupedStream groupedStream = ...; + KGroupedTable groupedTable = ...; + + // Java 8+ examples, using lambda expressions + + // Aggregating a KGroupedStream (note how the value type changes from String to Long) + KTable aggregatedStream = groupedStream.aggregate( + () -> 0L, /* initializer */ + (aggKey, newValue, aggValue) -> aggValue + newValue.length(), /* adder */ + Materialized.>as("aggregated-stream-store") /* state store name */ + .withValueSerde(Serdes.Long()); /* serde for aggregate value */ + + // Aggregating a KGroupedTable (note how the value type changes from String to Long) + KTable aggregatedTable = groupedTable.aggregate( + () -> 0L, /* initializer */ + (aggKey, newValue, aggValue) -> aggValue + newValue.length(), /* adder */ + (aggKey, oldValue, aggValue) -> aggValue - oldValue.length(), /* subtractor */ + Materialized.>as("aggregated-table-store") /* state store name */ + .withValueSerde(Serdes.Long()) /* serde for aggregate value */ + + + // Java 7 examples + + // Aggregating a KGroupedStream (note how the value type changes from String to Long) + KTable aggregatedStream = groupedStream.aggregate( + new Initializer() { /* initializer */ + @Override + public Long apply() { + return 0L; + } + }, + new Aggregator() { /* adder */ + @Override + public Long apply(byte[] aggKey, String newValue, Long aggValue) { + return aggValue + newValue.length(); + } + }, + Materialized.as("aggregated-stream-store") + .withValueSerde(Serdes.Long()); + + // Aggregating a KGroupedTable (note how the value type changes from String to Long) + KTable aggregatedTable = groupedTable.aggregate( + new Initializer() { /* initializer */ + @Override + public Long apply() { + return 0L; + } + }, + new Aggregator() { /* adder */ + @Override + public Long apply(byte[] aggKey, String newValue, Long aggValue) { + return aggValue + newValue.length(); + } + }, + new Aggregator() { /* subtractor */ + @Override + public Long apply(byte[] aggKey, String oldValue, Long aggValue) { + return aggValue - oldValue.length(); + } + }, + Materialized.as("aggregated-stream-store") + .withValueSerde(Serdes.Long()); + +Detailed behavior of `KGroupedStream`: + + * Input records with `null` keys are ignored. + * When a record key is received for the first time, the initializer is called (and called before the adder). + * Whenever a record with a non-`null` value is received, the adder is called. + +Detailed behavior of `KGroupedTable`: + + * Input records with `null` keys are ignored. + * When a record key is received for the first time, the initializer is called (and called before the adder and subtractor). Note that, in contrast to `KGroupedStream`, over time the initializer may be called more than once for a key as a result of having received input tombstone records for that key (see below). + * When the first non-`null` value is received for a key (e.g., INSERT), then only the adder is called. + * When subsequent non-`null` values are received for a key (e.g., UPDATE), then (1) the subtractor is called with the old value as stored in the table and (2) the adder is called with the new value of the input record that was just received. The subtractor is guaranteed to be called before the adder if the extracted grouping key of the old and new value is the same. The detection of this case depends on the correct implementation of the equals() method of the extracted key type. Otherwise, the order of execution for the subtractor and adder is not defined. + * When a tombstone record - i.e. a record with a `null` value - is received for a key (e.g., DELETE), then only the subtractor is called. Note that, whenever the subtractor returns a `null` value itself, then the corresponding key is removed from the resulting `KTable`. If that happens, any next input record for that key will trigger the initializer again. + +See the example at the bottom of this section for a visualization of the aggregation semantics. +**Aggregate (windowed)** + + * KGroupedStream -> KTable + +| **Windowed aggregation.** Aggregates the values of records, per window, by the grouped key. Aggregating is a generalization of `reduce` and allows, for example, the aggregate value to have a different type than the input values. ([TimeWindowedKStream details](/37/javadoc/org/apache/kafka/streams/kstream/TimeWindowedKStream.html), [SessionWindowedKStream details](/37/javadoc/org/apache/kafka/streams/kstream/SessionWindowedKStream.html)) You must provide an initializer (e.g., `aggValue = 0`), "adder" aggregator (e.g., `aggValue + curValue`), and a window. When windowing based on sessions, you must additionally provide a "session merger" aggregator (e.g., `mergedAggValue = leftAggValue + rightAggValue`). The windowed `aggregate` turns a `TimeWindowedKStream` or `SessionWindowedKStream` into a windowed `KTable, V>`. Several variants of `aggregate` exist, see Javadocs for details. + + + import java.time.Duration; + KGroupedStream groupedStream = ...; + + // Java 8+ examples, using lambda expressions + + // Aggregating with time-based windowing (here: with 5-minute tumbling windows) + KTable, Long> timeWindowedAggregatedStream = groupedStream.windowedBy(Duration.ofMinutes(5)) + .aggregate( + () -> 0L, /* initializer */ + (aggKey, newValue, aggValue) -> aggValue + newValue, /* adder */ + Materialized.>as("time-windowed-aggregated-stream-store") /* state store name */ + .withValueSerde(Serdes.Long())); /* serde for aggregate value */ + + // Aggregating with time-based windowing (here: with 5-minute sliding windows and 30-minute grace period) + KTable, Long> timeWindowedAggregatedStream = groupedStream.windowedBy(SlidingWindows.ofTimeDifferenceAndGrace(Duration.ofMinutes(5), Duration.ofMinutes(30))) + .aggregate( + () -> 0L, /* initializer */ + (aggKey, newValue, aggValue) -> aggValue + newValue, /* adder */ + Materialized.>as("time-windowed-aggregated-stream-store") /* state store name */ + .withValueSerde(Serdes.Long())); /* serde for aggregate value */ + + // Aggregating with session-based windowing (here: with an inactivity gap of 5 minutes) + KTable, Long> sessionizedAggregatedStream = groupedStream.windowedBy(SessionWindows.ofInactivityGapWithNoGrace(Duration.ofMinutes(5)). + aggregate( + () -> 0L, /* initializer */ + (aggKey, newValue, aggValue) -> aggValue + newValue, /* adder */ + (aggKey, leftAggValue, rightAggValue) -> leftAggValue + rightAggValue, /* session merger */ + Materialized.>as("sessionized-aggregated-stream-store") /* state store name */ + .withValueSerde(Serdes.Long())); /* serde for aggregate value */ + + // Java 7 examples + + // Aggregating with time-based windowing (here: with 5-minute tumbling windows) + KTable, Long> timeWindowedAggregatedStream = groupedStream.windowedBy(Duration.ofMinutes(5)) + .aggregate( + new Initializer() { /* initializer */ + @Override + public Long apply() { + return 0L; + } + }, + new Aggregator() { /* adder */ + @Override + public Long apply(String aggKey, Long newValue, Long aggValue) { + return aggValue + newValue; + } + }, + Materialized.>as("time-windowed-aggregated-stream-store") + .withValueSerde(Serdes.Long())); + + // Aggregating with session-based windowing (here: with an inactivity gap of 5 minutes) + KTable, Long> sessionizedAggregatedStream = groupedStream.windowedBy(SessionWindows.ofInactivityGapWithNoGrace(Duration.ofMinutes(5)). + aggregate( + new Initializer() { /* initializer */ + @Override + public Long apply() { + return 0L; + } + }, + new Aggregator() { /* adder */ + @Override + public Long apply(String aggKey, Long newValue, Long aggValue) { + return aggValue + newValue; + } + }, + new Merger() { /* session merger */ + @Override + public Long apply(String aggKey, Long leftAggValue, Long rightAggValue) { + return rightAggValue + leftAggValue; + } + }, + Materialized.>as("sessionized-aggregated-stream-store") + .withValueSerde(Serdes.Long())); + +Detailed behavior: + + * The windowed aggregate behaves similar to the rolling aggregate described above. The additional twist is that the behavior applies _per window_. + * Input records with `null` keys are ignored in general. + * When a record key is received for the first time for a given window, the initializer is called (and called before the adder). + * Whenever a record with a non-`null` value is received for a given window, the adder is called. + * When using session windows: the session merger is called whenever two sessions are being merged. + +See the example at the bottom of this section for a visualization of the aggregation semantics. +**Count** + + * KGroupedStream -> KTable + * KGroupedTable -> KTable + +| **Rolling aggregation.** Counts the number of records by the grouped key. ([KGroupedStream details](/37/javadoc/org/apache/kafka/streams/kstream/KGroupedStream.html), [KGroupedTable details](/37/javadoc/org/apache/kafka/streams/kstream/KGroupedTable.html)) Several variants of `count` exist, see Javadocs for details. + + + KGroupedStream groupedStream = ...; + KGroupedTable groupedTable = ...; + + // Counting a KGroupedStream + KTable aggregatedStream = groupedStream.count(); + + // Counting a KGroupedTable + KTable aggregatedTable = groupedTable.count(); + +Detailed behavior for `KGroupedStream`: + + * Input records with `null` keys or values are ignored. + +Detailed behavior for `KGroupedTable`: + + * Input records with `null` keys are ignored. Records with `null` values are not ignored but interpreted as "tombstones" for the corresponding key, which indicate the deletion of the key from the table. + + +**Count (windowed)** + + * KGroupedStream -> KTable + +| **Windowed aggregation.** Counts the number of records, per window, by the grouped key. ([TimeWindowedKStream details](/37/javadoc/org/apache/kafka/streams/kstream/TimeWindowedKStream.html), [SessionWindowedKStream details](/37/javadoc/org/apache/kafka/streams/kstream/SessionWindowedKStream.html)) The windowed `count` turns a `TimeWindowedKStream` or `SessionWindowedKStream` into a windowed `KTable, V>`. Several variants of `count` exist, see Javadocs for details. + + + import java.time.Duration; + KGroupedStream groupedStream = ...; + + // Counting a KGroupedStream with time-based windowing (here: with 5-minute tumbling windows) + KTable, Long> aggregatedStream = groupedStream.windowedBy( + TimeWindows.ofSizeWithNoGrace(Duration.ofMinutes(5))) /* time-based window */ + .count(); + + // Counting a KGroupedStream with time-based windowing (here: with 5-minute sliding windows and 30-minute grace period) + KTable, Long> aggregatedStream = groupedStream.windowedBy( + SlidingWindows.ofTimeDifferenceAndGrace(Duration.ofMinutes(5), Duration.ofMinutes(30))) /* time-based window */ + .count(); + + // Counting a KGroupedStream with session-based windowing (here: with 5-minute inactivity gaps) + KTable, Long> aggregatedStream = groupedStream.windowedBy( + SessionWindows.ofInactivityGapWithNoGrace(Duration.ofMinutes(5))) /* session window */ + .count(); + +Detailed behavior: + + * Input records with `null` keys or values are ignored. + + +**Reduce** + + * KGroupedStream -> KTable + * KGroupedTable -> KTable + +| **Rolling aggregation.** Combines the values of (non-windowed) records by the grouped key. The current record value is combined with the last reduced value, and a new reduced value is returned. The result value type cannot be changed, unlike `aggregate`. ([KGroupedStream details](/37/javadoc/org/apache/kafka/streams/kstream/KGroupedStream.html), [KGroupedTable details](/37/javadoc/org/apache/kafka/streams/kstream/KGroupedTable.html)) When reducing a _grouped stream_ , you must provide an "adder" reducer (e.g., `aggValue + curValue`). When reducing a _grouped table_ , you must additionally provide a "subtractor" reducer (e.g., `aggValue - oldValue`). Several variants of `reduce` exist, see Javadocs for details. + + + KGroupedStream groupedStream = ...; + KGroupedTable groupedTable = ...; + + // Java 8+ examples, using lambda expressions + + // Reducing a KGroupedStream + KTable aggregatedStream = groupedStream.reduce( + (aggValue, newValue) -> aggValue + newValue /* adder */); + + // Reducing a KGroupedTable + KTable aggregatedTable = groupedTable.reduce( + (aggValue, newValue) -> aggValue + newValue, /* adder */ + (aggValue, oldValue) -> aggValue - oldValue /* subtractor */); + + + // Java 7 examples + + // Reducing a KGroupedStream + KTable aggregatedStream = groupedStream.reduce( + new Reducer() { /* adder */ + @Override + public Long apply(Long aggValue, Long newValue) { + return aggValue + newValue; + } + }); + + // Reducing a KGroupedTable + KTable aggregatedTable = groupedTable.reduce( + new Reducer() { /* adder */ + @Override + public Long apply(Long aggValue, Long newValue) { + return aggValue + newValue; + } + }, + new Reducer() { /* subtractor */ + @Override + public Long apply(Long aggValue, Long oldValue) { + return aggValue - oldValue; + } + }); + +Detailed behavior for `KGroupedStream`: + + * Input records with `null` keys are ignored in general. + * When a record key is received for the first time, then the value of that record is used as the initial aggregate value. + * Whenever a record with a non-`null` value is received, the adder is called. + +Detailed behavior for `KGroupedTable`: + + * Input records with `null` keys are ignored in general. + * When a record key is received for the first time, then the value of that record is used as the initial aggregate value. Note that, in contrast to `KGroupedStream`, over time this initialization step may happen more than once for a key as a result of having received input tombstone records for that key (see below). + * When the first non-`null` value is received for a key (e.g., INSERT), then only the adder is called. + * When subsequent non-`null` values are received for a key (e.g., UPDATE), then (1) the subtractor is called with the old value as stored in the table and (2) the adder is called with the new value of the input record that was just received. The subtractor is guaranteed be called before the adder if the extracted grouping key of the old and new value is the same. The detection of this case depends on the correct implementation of the equals() method of the extracted key type. Otherwise, the order of execution for the subtractor and adder is not defined. + * When a tombstone record - i.e. a record with a `null` value - is received for a key (e.g., DELETE), then only the subtractor is called. Note that, whenever the subtractor returns a `null` value itself, then the corresponding key is removed from the resulting `KTable`. If that happens, any next input record for that key will re-initialize its aggregate value. + +See the example at the bottom of this section for a visualization of the aggregation semantics. +**Reduce (windowed)** + + * KGroupedStream -> KTable + +| **Windowed aggregation.** Combines the values of records, per window, by the grouped key. The current record value is combined with the last reduced value, and a new reduced value is returned. Records with `null` key or value are ignored. The result value type cannot be changed, unlike `aggregate`. ([TimeWindowedKStream details](/37/javadoc/org/apache/kafka/streams/kstream/TimeWindowedKStream.html), [SessionWindowedKStream details](/37/javadoc/org/apache/kafka/streams/kstream/SessionWindowedKStream.html)) The windowed `reduce` turns a turns a `TimeWindowedKStream` or a `SessionWindowedKStream` into a windowed `KTable, V>`. Several variants of `reduce` exist, see Javadocs for details. + + + import java.time.Duration; + KGroupedStream groupedStream = ...; + + // Java 8+ examples, using lambda expressions + + // Aggregating with time-based windowing (here: with 5-minute tumbling windows) + KTable, Long> timeWindowedAggregatedStream = groupedStream.windowedBy( + TimeWindows.ofSizeWithNoGrace(Duration.ofMinutes(5)) /* time-based window */) + .reduce( + (aggValue, newValue) -> aggValue + newValue /* adder */ + ); + + // Aggregating with time-based windowing (here: with 5-minute sliding windows and 30-minute grace) + KTable, Long> timeWindowedAggregatedStream = groupedStream.windowedBy( + SlidingWindows.ofTimeDifferenceAndGrace(Duration.ofMinutes(5), Duration.ofMinutes(30))) /* time-based window */) + .reduce( + (aggValue, newValue) -> aggValue + newValue /* adder */ + ); + + // Aggregating with session-based windowing (here: with an inactivity gap of 5 minutes) + KTable, Long> sessionzedAggregatedStream = groupedStream.windowedBy( + SessionWindows.ofInactivityGapWithNoGrace(Duration.ofMinutes(5))) /* session window */ + .reduce( + (aggValue, newValue) -> aggValue + newValue /* adder */ + ); + + + // Java 7 examples + + // Aggregating with time-based windowing (here: with 5-minute tumbling windows) + KTable, Long> timeWindowedAggregatedStream = groupedStream.windowedBy( + TimeWindows.ofSizeWithNoGrace(Duration.ofMinutes(5)) /* time-based window */) + .reduce( + new Reducer() { /* adder */ + @Override + public Long apply(Long aggValue, Long newValue) { + return aggValue + newValue; + } + }); + + // Aggregating with session-based windowing (here: with an inactivity gap of 5 minutes) + KTable, Long> timeWindowedAggregatedStream = groupedStream.windowedBy( + SessionWindows.ofInactivityGapWithNoGrace(Duration.ofMinutes(5))) /* session window */ + .reduce( + new Reducer() { /* adder */ + @Override + public Long apply(Long aggValue, Long newValue) { + return aggValue + newValue; + } + }); + +Detailed behavior: + + * The windowed reduce behaves similar to the rolling reduce described above. The additional twist is that the behavior applies _per window_. + * Input records with `null` keys are ignored in general. + * When a record key is received for the first time for a given window, then the value of that record is used as the initial aggregate value. + * Whenever a record with a non-`null` value is received for a given window, the adder is called. + +See the example at the bottom of this section for a visualization of the aggregation semantics. + +**Example of semantics for stream aggregations:** A `KGroupedStream` -> `KTable` example is shown below. The streams and the table are initially empty. Bold font is used in the column for "KTable `aggregated`" to highlight changed state. An entry such as `(hello, 1)` denotes a record with key `hello` and value `1`. To improve the readability of the semantics table you can assume that all records are processed in timestamp order. + + + // Key: word, value: count + KStream wordCounts = ...; + + KGroupedStream groupedStream = wordCounts + .groupByKey(Grouped.with(Serdes.String(), Serdes.Integer())); + + KTable aggregated = groupedStream.aggregate( + () -> 0, /* initializer */ + (aggKey, newValue, aggValue) -> aggValue + newValue, /* adder */ + Materialized.as("aggregated-stream-store" /* state store name */) + .withKeySerde(Serdes.String()) /* key serde */ + .withValueSerde(Serdes.Integer()); /* serde for aggregate value */ + +**Note** + +**Impact of record caches** : For illustration purposes, the column "KTable `aggregated`" below shows the table's state changes over time in a very granular way. In practice, you would observe state changes in such a granular way only when [record caches](memory-mgmt.html#streams-developer-guide-memory-management-record-cache) are disabled (default: enabled). When record caches are enabled, what might happen for example is that the output results of the rows with timestamps 4 and 5 would be [compacted](memory-mgmt.html#streams-developer-guide-memory-management-record-cache), and there would only be a single state update for the key `kafka` in the KTable (here: from `(kafka 1)` directly to `(kafka, 3)`. Typically, you should only disable record caches for testing or debugging purposes - under normal circumstances it is better to leave record caches enabled. + + | KStream `wordCounts` | KGroupedStream `groupedStream` | KTable `aggregated` +---|---|---|--- +Timestamp | Input record | Grouping | Initializer | Adder | State +1 | (hello, 1) | (hello, 1) | 0 (for hello) | (hello, 0 + 1) | **(hello, 1)** +2 | (kafka, 1) | (kafka, 1) | 0 (for kafka) | (kafka, 0 + 1) | (hello, 1) **(kafka, 1)** +3 | (streams, 1) | (streams, 1) | 0 (for streams) | (streams, 0 + 1) | (hello, 1) (kafka, 1) **(streams, 1)** +4 | (kafka, 1) | (kafka, 1) | | (kafka, 1 + 1) | (hello, 1) (kafka, **2**) (streams, 1) +5 | (kafka, 1) | (kafka, 1) | | (kafka, 2 + 1) | (hello, 1) (kafka, **3**) (streams, 1) +6 | (streams, 1) | (streams, 1) | | (streams, 1 + 1) | (hello, 1) (kafka, 3) (streams, **2**) + +**Example of semantics for table aggregations:** A `KGroupedTable` -> `KTable` example is shown below. The tables are initially empty. Bold font is used in the column for "KTable `aggregated`" to highlight changed state. An entry such as `(hello, 1)` denotes a record with key `hello` and value `1`. To improve the readability of the semantics table you can assume that all records are processed in timestamp order. + + + // Key: username, value: user region (abbreviated to "E" for "Europe", "A" for "Asia") + KTable userProfiles = ...; + + // Re-group `userProfiles`. Don't read too much into what the grouping does: + // its prime purpose in this example is to show the *effects* of the grouping + // in the subsequent aggregation. + KGroupedTable groupedTable = userProfiles + .groupBy((user, region) -> KeyValue.pair(region, user.length()), Serdes.String(), Serdes.Integer()); + + KTable aggregated = groupedTable.aggregate( + () -> 0, /* initializer */ + (aggKey, newValue, aggValue) -> aggValue + newValue, /* adder */ + (aggKey, oldValue, aggValue) -> aggValue - oldValue, /* subtractor */ + Materialized.as("aggregated-table-store" /* state store name */) + .withKeySerde(Serdes.String()) /* key serde */ + .withValueSerde(Serdes.Integer()); /* serde for aggregate value */ + +**Note** + +**Impact of record caches** : For illustration purposes, the column "KTable `aggregated`" below shows the table's state changes over time in a very granular way. In practice, you would observe state changes in such a granular way only when [record caches](memory-mgmt.html#streams-developer-guide-memory-management-record-cache) are disabled (default: enabled). When record caches are enabled, what might happen for example is that the output results of the rows with timestamps 4 and 5 would be [compacted](memory-mgmt.html#streams-developer-guide-memory-management-record-cache), and there would only be a single state update for the key `kafka` in the KTable (here: from `(kafka 1)` directly to `(kafka, 3)`. Typically, you should only disable record caches for testing or debugging purposes - under normal circumstances it is better to leave record caches enabled. + + | KTable `userProfiles` | KGroupedTable `groupedTable` | KTable `aggregated` +---|---|---|--- +Timestamp | Input record | Interpreted as | Grouping | Initializer | Adder | Subtractor | State +1 | (alice, E) | INSERT alice | (E, 5) | 0 (for E) | (E, 0 + 5) | | **(E, 5)** +2 | (bob, A) | INSERT bob | (A, 3) | 0 (for A) | (A, 0 + 3) | | **(A, 3)** (E, 5) +3 | (charlie, A) | INSERT charlie | (A, 7) | | (A, 3 + 7) | | (A, **10**) (E, 5) +4 | (alice, A) | UPDATE alice | (A, 5) | | (A, 10 + 5) | (E, 5 - 5) | (A, **15**) (E, **0**) +5 | (charlie, null) | DELETE charlie | (null, 7) | | | (A, 15 - 7) | (A, **8**) (E, 0) +6 | (null, E) | _ignored_ | | | | | (A, 8) (E, 0) +7 | (bob, E) | UPDATE bob | (E, 3) | | (E, 0 + 3) | (A, 8 - 3) | (A, **5**) (E, **3**) + +### Joining + +Streams and tables can also be joined. Many stream processing applications in practice are coded as streaming joins. For example, applications backing an online shop might need to access multiple, updating database tables (e.g. sales prices, inventory, customer information) in order to enrich a new data record (e.g. customer transaction) with context information. That is, scenarios where you need to perform table lookups at very large scale and with a low processing latency. Here, a popular pattern is to make the information in the databases available in Kafka through so-called _change data capture_ in combination with [Kafka's Connect API](../../#connect), and then implementing applications that leverage the Streams API to perform very fast and efficient local joins of such tables and streams, rather than requiring the application to make a query to a remote database over the network for each record. In this example, the KTable concept in Kafka Streams would enable you to track the latest state (e.g., snapshot) of each table in a local state store, thus greatly reducing the processing latency as well as reducing the load of the remote databases when doing such streaming joins. + +The following join operations are supported, see also the diagram in the overview section of Stateful Transformations. Depending on the operands, joins are either windowed joins or non-windowed joins. + +Join operands | Type | (INNER) JOIN | LEFT JOIN | OUTER JOIN +---|---|---|---|--- +KStream-to-KStream | Windowed | Supported | Supported | Supported +KTable-to-KTable | Non-windowed | Supported | Supported | Supported +KTable-to-KTable Foreign-Key Join | Non-windowed | Supported | Supported | Not Supported +KStream-to-KTable | Non-windowed | Supported | Supported | Not Supported +KStream-to-GlobalKTable | Non-windowed | Supported | Supported | Not Supported +KTable-to-GlobalKTable | N/A | Not Supported | Not Supported | Not Supported + +Each case is explained in more detail in the subsequent sections. + +#### Join co-partitioning requirements + +For equi-joins, input data must be co-partitioned when joining. This ensures that input records with the same key from both sides of the join, are delivered to the same stream task during processing. **It is your responsibility to ensure data co-partitioning when joining**. Co-partitioning is not required when performing KTable-KTable Foreign-Key joins and Global KTable joins. + +The requirements for data co-partitioning are: + + * The input topics of the join (left side and right side) must have the **same number of partitions**. + * All applications that _write_ to the input topics must have the **same partitioning strategy** so that records with the same key are delivered to same partition number. In other words, the keyspace of the input data must be distributed across partitions in the same manner. This means that, for example, applications that use Kafka's [Java Producer API](../../#producerapi) must use the same partitioner (cf. the producer setting `"partitioner.class"` aka `ProducerConfig.PARTITIONER_CLASS_CONFIG`), and applications that use the Kafka's Streams API must use the same `StreamPartitioner` for operations such as `KStream#to()`. The good news is that, if you happen to use the default partitioner-related settings across all applications, you do not need to worry about the partitioning strategy. + + + +Why is data co-partitioning required? Because KStream-KStream, KTable-KTable, and KStream-KTable joins are performed based on the keys of records (e.g., `leftRecord.key == rightRecord.key`), it is required that the input streams/tables of a join are co-partitioned by key. + +There are two exceptions where co-partitioning is not required. For KStream-GlobalKTable joins joins, co-partitioning is not required because _all_ partitions of the `GlobalKTable`'s underlying changelog stream are made available to each `KafkaStreams` instance. That is, each instance has a full copy of the changelog stream. Further, a `KeyValueMapper` allows for non-key based joins from the `KStream` to the `GlobalKTable`. KTable-KTable Foreign-Key joins also do not require co-partitioning. Kafka Streams internally ensures co-partitioning for Foreign-Key joins. + +**Note** + +**Kafka Streams partly verifies the co-partitioning requirement:** During the partition assignment step, i.e. at runtime, Kafka Streams verifies whether the number of partitions for both sides of a join are the same. If they are not, a `TopologyBuilderException` (runtime exception) is being thrown. Note that Kafka Streams cannot verify whether the partitioning strategy matches between the input streams/tables of a join - it is up to the user to ensure that this is the case. + +**Ensuring data co-partitioning:** If the inputs of a join are not co-partitioned yet, you must ensure this manually. You may follow a procedure such as outlined below. It is recommended to repartition the topic with fewer partitions to match the larger partition number of avoid bottlenecks. Technically it would also be possible to repartition the topic with more partitions to the smaller partition number. For stream-table joins, it's recommended to repartition the KStream because repartitioning a KTable might result in a second state store. For table-table joins, you might also consider to size of the KTables and repartition the smaller KTable. + + 1. Identify the input KStream/KTable in the join whose underlying Kafka topic has the smaller number of partitions. Let's call this stream/table "SMALLER", and the other side of the join "LARGER". To learn about the number of partitions of a Kafka topic you can use, for example, the CLI tool `bin/kafka-topics` with the `--describe` option. + + 2. Within your application, re-partition the data of "SMALLER". You must ensure that, when repartitioning the data with `repartition`, the same partitioner is used as for "LARGER". + +> * If "SMALLER" is a KStream: `KStream#repartition(Repartitioned.numberOfPartitions(...))`. +> * If "SMALLER" is a KTable: `KTable#toStream#repartition(Repartitioned.numberOfPartitions(...).toTable())`. + + 3. Within your application, perform the join between "LARGER" and the new stream/table. + + + + +#### KStream-KStream Join + +KStream-KStream joins are always windowed joins, because otherwise the size of the internal state store used to perform the join - e.g., a sliding window or "buffer" - would grow indefinitely. For stream-stream joins it's important to highlight that a new input record on one side will produce a join output _for each_ matching record on the other side, and there can be _multiple_ such matching records in a given join window (cf. the row with timestamp 15 in the join semantics table below, for example). + +Join output records are effectively created as follows, leveraging the user-supplied `ValueJoiner`: + + + KeyValue leftRecord = ...; + KeyValue rightRecord = ...; + ValueJoiner joiner = ...; + + KeyValue joinOutputRecord = KeyValue.pair( + leftRecord.key, /* by definition, leftRecord.key == rightRecord.key */ + joiner.apply(leftRecord.value, rightRecord.value) + ); + +Transformation | Description +---|--- +**Inner Join (windowed)** + + * (KStream, KStream) -> KStream + +| Performs an INNER JOIN of this stream with another stream. Even though this operation is windowed, the joined stream will be of type `KStream` rather than `KStream, ...>`. [(details)](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#join-org.apache.kafka.streams.kstream.KStream-org.apache.kafka.streams.kstream.ValueJoiner-org.apache.kafka.streams.kstream.JoinWindows-) **Data must be co-partitioned** : The input data for both sides must be co-partitioned. **Causes data re-partitioning of a stream if and only if the stream was marked for re-partitioning (if both are marked, both are re-partitioned).** Several variants of `join` exists, see the Javadocs for details. + + + import java.time.Duration; + KStream left = ...; + KStream right = ...; + + // Java 8+ example, using lambda expressions + KStream joined = left.join(right, + (leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue, /* ValueJoiner */ + JoinWindows.ofTimeDifferenceWithNoGrace(Duration.ofMinutes(5)), + Joined.with( + Serdes.String(), /* key */ + Serdes.Long(), /* left value */ + Serdes.Double()) /* right value */ + ); + + // Java 7 example + KStream joined = left.join(right, + new ValueJoiner() { + @Override + public String apply(Long leftValue, Double rightValue) { + return "left=" + leftValue + ", right=" + rightValue; + } + }, + JoinWindows.ofTimeDifferenceWithNoGrace(Duration.ofMinutes(5)), + Joined.with( + Serdes.String(), /* key */ + Serdes.Long(), /* left value */ + Serdes.Double()) /* right value */ + ); + +Detailed behavior: + + * The join is _key-based_ , i.e. with the join predicate `leftRecord.key == rightRecord.key`, and _window-based_ , i.e. two input records are joined if and only if their timestamps are "close" to each other as defined by the user-supplied `JoinWindows`, i.e. the window defines an additional join predicate over the record timestamps. + * The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied `ValueJoiner` will be called to produce join output records. + +> * Input records with a `null` key or a `null` value are ignored and do not trigger the join. + + +See the semantics overview at the bottom of this section for a detailed description. +**Left Join (windowed)** + + * (KStream, KStream) -> KStream + +| Performs a LEFT JOIN of this stream with another stream. Even though this operation is windowed, the joined stream will be of type `KStream` rather than `KStream, ...>`. [(details)](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#leftJoin-org.apache.kafka.streams.kstream.KStream-org.apache.kafka.streams.kstream.ValueJoiner-org.apache.kafka.streams.kstream.JoinWindows-) **Data must be co-partitioned** : The input data for both sides must be co-partitioned. **Causes data re-partitioning of a stream if and only if the stream was marked for re-partitioning (if both are marked, both are re-partitioned).** Several variants of `leftJoin` exists, see the Javadocs for details. + + + import java.time.Duration; + KStream left = ...; + KStream right = ...; + + // Java 8+ example, using lambda expressions + KStream joined = left.leftJoin(right, + (leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue, /* ValueJoiner */ + JoinWindows.ofTimeDifferenceWithNoGrace(Duration.ofMinutes(5)), + Joined.with( + Serdes.String(), /* key */ + Serdes.Long(), /* left value */ + Serdes.Double()) /* right value */ + ); + + // Java 7 example + KStream joined = left.leftJoin(right, + new ValueJoiner() { + @Override + public String apply(Long leftValue, Double rightValue) { + return "left=" + leftValue + ", right=" + rightValue; + } + }, + JoinWindows.ofTimeDifferenceWithNoGrace(Duration.ofMinutes(5)), + Joined.with( + Serdes.String(), /* key */ + Serdes.Long(), /* left value */ + Serdes.Double()) /* right value */ + ); + +Detailed behavior: + + * The join is _key-based_ , i.e. with the join predicate `leftRecord.key == rightRecord.key`, and _window-based_ , i.e. two input records are joined if and only if their timestamps are "close" to each other as defined by the user-supplied `JoinWindows`, i.e. the window defines an additional join predicate over the record timestamps. + * The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied `ValueJoiner` will be called to produce join output records. + +> * Input records with a `null` value are ignored and do not trigger the join. + + * For each input record on the left side that does not have any match on the right side, the `ValueJoiner` will be called with `ValueJoiner#apply(leftRecord.value, null)`; this explains the row with timestamp=60 and timestampe=80 in the table below, which lists `[E, null]` and `[F, null]`in the LEFT JOIN column. Note that these left results are emitted after the specified grace period passed. **Caution:** using the deprecated `JoinWindows.of(...).grace(...)` API might result in eagerly emitted spurious left results. + +See the semantics overview at the bottom of this section for a detailed description. +**Outer Join (windowed)** + + * (KStream, KStream) -> KStream + +| Performs an OUTER JOIN of this stream with another stream. Even though this operation is windowed, the joined stream will be of type `KStream` rather than `KStream, ...>`. [(details)](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#outerJoin-org.apache.kafka.streams.kstream.KStream-org.apache.kafka.streams.kstream.ValueJoiner-org.apache.kafka.streams.kstream.JoinWindows-) **Data must be co-partitioned** : The input data for both sides must be co-partitioned. **Causes data re-partitioning of a stream if and only if the stream was marked for re-partitioning (if both are marked, both are re-partitioned).** Several variants of `outerJoin` exists, see the Javadocs for details. + + + import java.time.Duration; + KStream left = ...; + KStream right = ...; + + // Java 8+ example, using lambda expressions + KStream joined = left.outerJoin(right, + (leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue, /* ValueJoiner */ + JoinWindows.ofTimeDifferenceWithNoGrace(Duration.ofMinutes(5)), + Joined.with( + Serdes.String(), /* key */ + Serdes.Long(), /* left value */ + Serdes.Double()) /* right value */ + ); + + // Java 7 example + KStream joined = left.outerJoin(right, + new ValueJoiner() { + @Override + public String apply(Long leftValue, Double rightValue) { + return "left=" + leftValue + ", right=" + rightValue; + } + }, + JoinWindows.ofTimeDifferenceWithNoGrace(Duration.ofMinutes(5)), + Joined.with( + Serdes.String(), /* key */ + Serdes.Long(), /* left value */ + Serdes.Double()) /* right value */ + ); + +Detailed behavior: + + * The join is _key-based_ , i.e. with the join predicate `leftRecord.key == rightRecord.key`, and _window-based_ , i.e. two input records are joined if and only if their timestamps are "close" to each other as defined by the user-supplied `JoinWindows`, i.e. the window defines an additional join predicate over the record timestamps. + * The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied `ValueJoiner` will be called to produce join output records. + +> * Input records with a `null` value are ignored and do not trigger the join. + + * For each input record on one side that does not have any match on the other side, the `ValueJoiner` will be called with `ValueJoiner#apply(leftRecord.value, null)` or `ValueJoiner#apply(null, rightRecord.value)`, respectively; this explains the row with timestamp=60, timestamp=80, and timestamp=100 in the table below, which lists `[E, null]`, `[F, null]`, and `[null, f]` in the OUTER JOIN column. Note that these left and right results are emitted after the specified grace period passed. **Caution:** using the deprecated `JoinWindows.of(...).grace(...)` API might result in eagerly emitted spurious left/right results. + +See the semantics overview at the bottom of this section for a detailed description. + +**Semantics of stream-stream joins:** The semantics of the various stream-stream join variants are explained below. To improve the readability of the table, assume that (1) all records have the same key (and thus the key in the table is omitted), and (2) all records are processed in timestamp order. We assume a join window size of 10 seconds with a grace period of 5 seconds. + +**Note:** If you use the old and now deprecated API to specify the grace period, i.e., `JoinWindows.of(...).grace(...)`, left/outer join results are emitted eagerly, and the observed result might differ from the result shown below. + +The columns INNER JOIN, LEFT JOIN, and OUTER JOIN denote what is passed as arguments to the user-supplied [ValueJoiner](/37/javadoc/org/apache/kafka/streams/kstream/ValueJoiner.html) for the `join`, `leftJoin`, and `outerJoin` methods, respectively, whenever a new input record is received on either side of the join. An empty table cell denotes that the `ValueJoiner` is not called at all. + +Timestamp | Left (KStream) | Right (KStream) | (INNER) JOIN | LEFT JOIN | OUTER JOIN +---|---|---|---|---|--- +1 | null | | | | +2 | | null | | | +3 | A | | | | +4 | | a | [A, a] | [A, a] | [A, a] +5 | B | | [B, a] | [B, a] | [B, a] +6 | | b | [A, b], [B, b] | [A, b], [B, b] | [A, b], [B, b] +7 | null | | | | +8 | | null | | | +9 | C | | [C, a], [C, b] | [C, a], [C, b] | [C, a], [C, b] +10 | | c | [A, c], [B, c], [C, c] | [A, c], [B, c], [C, c] | [A, c], [B, c], [C, c] +11 | | null | | | +12 | null | | | | +13 | | null | | | +14 | | d | [A, d], [B, d], [C, d] | [A, d], [B, d], [C, d] | [A, d], [B, d], [C, d] +15 | D | | [D, a], [D, b], [D, c], [D, d] | [D, a], [D, b], [D, c], [D, d] | [D, a], [D, b], [D, c], [D, d] +... | | | | | +40 | E | | | | +... | | | | | +60 | F | | | [E, null] | [E, null] +... | | | | | +80 | | f | | [F, null] | [F, null] +... | | | | | +100 | G | | | | [null, f] + +#### KTable-KTable Equi-Join + +KTable-KTable equi-joins are always _non-windowed_ joins. They are designed to be consistent with their counterparts in relational databases. The changelog streams of both KTables are materialized into local state stores to represent the latest snapshot of their table duals. The join result is a new KTable that represents the changelog stream of the join operation. + +Join output records are effectively created as follows, leveraging the user-supplied `ValueJoiner`: + + + KeyValue leftRecord = ...; + KeyValue rightRecord = ...; + ValueJoiner joiner = ...; + + KeyValue joinOutputRecord = KeyValue.pair( + leftRecord.key, /* by definition, leftRecord.key == rightRecord.key */ + joiner.apply(leftRecord.value, rightRecord.value) + ); + +Transformation | Description +---|--- +**Inner Join** + + * (KTable, KTable) -> KTable + +| Performs an INNER JOIN of this table with another table. The result is an ever-updating KTable that represents the "current" result of the join. [(details)](/37/javadoc/org/apache/kafka/streams/kstream/KTable.html#join-org.apache.kafka.streams.kstream.KTable-org.apache.kafka.streams.kstream.ValueJoiner-) **Data must be co-partitioned** : The input data for both sides must be co-partitioned. + + + KTable left = ...; + KTable right = ...; + + // Java 8+ example, using lambda expressions + KTable joined = left.join(right, + (leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue /* ValueJoiner */ + ); + + // Java 7 example + KTable joined = left.join(right, + new ValueJoiner() { + @Override + public String apply(Long leftValue, Double rightValue) { + return "left=" + leftValue + ", right=" + rightValue; + } + }); + +Detailed behavior: + + * The join is _key-based_ , i.e. with the join predicate `leftRecord.key == rightRecord.key`. + * The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied `ValueJoiner` will be called to produce join output records. + +> * Input records with a `null` key are ignored and do not trigger the join. +> * Input records with a `null` value are interpreted as _tombstones_ for the corresponding key, which indicate the deletion of the key from the table. Tombstones do not trigger the join. When an input tombstone is received, then an output tombstone is forwarded directly to the join result KTable if required (i.e. only if the corresponding key actually exists already in the join result KTable). +> * When joining versioned tables, out-of-order input records, i.e., those for which another record from the same table, with the same key and a larger timestamp, has already been processed, are ignored and do not trigger the join. + + +See the semantics overview at the bottom of this section for a detailed description. +**Left Join** + + * (KTable, KTable) -> KTable + +| Performs a LEFT JOIN of this table with another table. [(details)](/37/javadoc/org/apache/kafka/streams/kstream/KTable.html#leftJoin-org.apache.kafka.streams.kstream.KTable-org.apache.kafka.streams.kstream.ValueJoiner-) **Data must be co-partitioned** : The input data for both sides must be co-partitioned. + + + KTable left = ...; + KTable right = ...; + + // Java 8+ example, using lambda expressions + KTable joined = left.leftJoin(right, + (leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue /* ValueJoiner */ + ); + + // Java 7 example + KTable joined = left.leftJoin(right, + new ValueJoiner() { + @Override + public String apply(Long leftValue, Double rightValue) { + return "left=" + leftValue + ", right=" + rightValue; + } + }); + +Detailed behavior: + + * The join is _key-based_ , i.e. with the join predicate `leftRecord.key == rightRecord.key`. + * The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied `ValueJoiner` will be called to produce join output records. + +> * Input records with a `null` key are ignored and do not trigger the join. +> * Input records with a `null` value are interpreted as _tombstones_ for the corresponding key, which indicate the deletion of the key from the table. Right-tombstones trigger the join, but left-tombstones don't: when an input tombstone is received, an output tombstone is forwarded directly to the join result KTable if required (i.e. only if the corresponding key actually exists already in the join result KTable). +> * When joining versioned tables, out-of-order input records, i.e., those for which another record from the same table, with the same key and a larger timestamp, has already been processed, are ignored and do not trigger the join. + + * For each input record on the left side that does not have any match on the right side, the `ValueJoiner` will be called with `ValueJoiner#apply(leftRecord.value, null)`; this explains the row with timestamp=3 in the table below, which lists `[A, null]` in the LEFT JOIN column. + +See the semantics overview at the bottom of this section for a detailed description. +**Outer Join** + + * (KTable, KTable) -> KTable + +| Performs an OUTER JOIN of this table with another table. [(details)](/37/javadoc/org/apache/kafka/streams/kstream/KTable.html#outerJoin-org.apache.kafka.streams.kstream.KTable-org.apache.kafka.streams.kstream.ValueJoiner-) **Data must be co-partitioned** : The input data for both sides must be co-partitioned. + + + KTable left = ...; + KTable right = ...; + + // Java 8+ example, using lambda expressions + KTable joined = left.outerJoin(right, + (leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue /* ValueJoiner */ + ); + + // Java 7 example + KTable joined = left.outerJoin(right, + new ValueJoiner() { + @Override + public String apply(Long leftValue, Double rightValue) { + return "left=" + leftValue + ", right=" + rightValue; + } + }); + +Detailed behavior: + + * The join is _key-based_ , i.e. with the join predicate `leftRecord.key == rightRecord.key`. + * The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied `ValueJoiner` will be called to produce join output records. + +> * Input records with a `null` key are ignored and do not trigger the join. +> * Input records with a `null` value are interpreted as _tombstones_ for the corresponding key, which indicate the deletion of the key from the table. Tombstones may trigger joins, depending on the content in the left and right tables. When an input tombstone is received, an output tombstone is forwarded directly to the join result KTable if required (i.e. only if the corresponding key actually exists already in the join result KTable). +> * When joining versioned tables, out-of-order input records, i.e., those for which another record from the same table, with the same key and a larger timestamp, has already been processed, are ignored and do not trigger the join. + + * For each input record on one side that does not have any match on the other side, the `ValueJoiner` will be called with `ValueJoiner#apply(leftRecord.value, null)` or `ValueJoiner#apply(null, rightRecord.value)`, respectively; this explains the rows with timestamp=3 and timestamp=7 in the table below, which list `[A, null]` and `[null, b]`, respectively, in the OUTER JOIN column. + +See the semantics overview at the bottom of this section for a detailed description. + +**Semantics of table-table equi-joins:** The semantics of the various table-table equi-join variants are explained below. To improve the readability of the table, you can assume that (1) all records have the same key (and thus the key in the table is omitted) and that (2) all records are processed in timestamp order. The columns INNER JOIN, LEFT JOIN, and OUTER JOIN denote what is passed as arguments to the user-supplied [ValueJoiner](/37/javadoc/org/apache/kafka/streams/kstream/ValueJoiner.html) for the `join`, `leftJoin`, and `outerJoin` methods, respectively, whenever a new input record is received on either side of the join. An empty table cell denotes that the `ValueJoiner` is not called at all. + +Timestamp | Left (KTable) | Right (KTable) | (INNER) JOIN | LEFT JOIN | OUTER JOIN +---|---|---|---|---|--- +1 | null | | | | +2 | | null | | | +3 | A | | | [A, null] | [A, null] +4 | | a | [A, a] | [A, a] | [A, a] +5 | B | | [B, a] | [B, a] | [B, a] +6 | | b | [B, b] | [B, b] | [B, b] +7 | null | | null | null | [null, b] +8 | | null | | | null +9 | C | | | [C, null] | [C, null] +10 | | c | [C, c] | [C, c] | [C, c] +11 | | null | null | [C, null] | [C, null] +12 | null | | | null | null +13 | | null | | | +14 | | d | | | [null, d] +15 | D | | [D, d] | [D, d] | [D, d] + +#### KTable-KTable Foreign-Key Join + +KTable-KTable foreign-key joins are always _non-windowed_ joins. Foreign-key joins are analogous to joins in SQL. As a rough example: + +` SELECT ... FROM {this KTable} JOIN {other KTable} ON {other.key} = {result of foreignKeyExtractor(this.value)} ... ` + +The output of the operation is a new KTable containing the join result. + +The changelog streams of both KTables are materialized into local state stores to represent the latest snapshot of their table duals. A foreign-key extractor function is applied to the left record, with a new intermediate record created and is used to lookup and join with the corresponding primary key on the right hand side table. The result is a new KTable that represents the changelog stream of the join operation. + +The left KTable can have multiple records which map to the same key on the right KTable. An update to a single left KTable entry may result in a single output event, provided the corresponding key exists in the right KTable. Consequently, a single update to a right KTable entry will result in an update for each record in the left KTable that has the same foreign key. + + + +Transformation | Description +---|--- +**Inner Join** + + * (KTable, KTable) -> KTable + +| Performs a foreign-key INNER JOIN of this table with another table. The result is an ever-updating KTable that represents the "current" result of the join. [(details)](/%7B%7Bversion%7D%7D/javadoc/org/apache/kafka/streams/kstream/KTable.html#join-org.apache.kafka.streams.kstream.KTable-org.apache.kafka.streams.kstream.ValueJoiner-) + + + KTable left = ...; + KTable right = ...; + //This foreignKeyExtractor simply uses the left-value to map to the right-key. + Function foreignKeyExtractor = (x) -> x; + + // Java 8+ example, using lambda expressions + KTable joined = left.join(right, foreignKeyExtractor, + (leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue /* ValueJoiner */ + ); + +Detailed behavior: + + * The join is _key-based_ , i.e. with the join predicate: + + foreignKeyExtractor.apply(leftRecord.value) == rightRecord.key + + * The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied `ValueJoiner` will be called to produce join output records. + +> * Records for which the `foreignKeyExtractor` produces `null` are ignored and do not trigger a join. If you want to join with `null` foreign keys, use a suitable sentinel value to do so (i.e. `"NULL"` for a String field, or `-1` for an auto-incrementing integer field). +> * Input records with a `null` value are interpreted as _tombstones_ for the corresponding key, which indicate the deletion of the key from the table. Tombstones do not trigger the join. When an input tombstone is received, then an output tombstone is forwarded directly to the join result KTable if required (i.e. only if the corresponding key actually exists already in the join result KTable). +> * When joining versioned tables, out-of-order input records, i.e., those for which another record from the same table, with the same key and a larger timestamp, has already been processed, are ignored and do not trigger the join. + + +See the semantics overview at the bottom of this section for a detailed description. +**Left Join** + + * (KTable, KTable) -> KTable + +| Performs a foreign-key LEFT JOIN of this table with another table. [(details)](/%7B%7Bversion%7D%7D/javadoc/org/apache/kafka/streams/kstream/KTable.html#leftJoin-org.apache.kafka.streams.kstream.KTable-org.apache.kafka.streams.kstream.ValueJoiner-) + + + KTable left = ...; + KTable right = ...; + //This foreignKeyExtractor simply uses the left-value to map to the right-key. + Function foreignKeyExtractor = (x) -> x; + + // Java 8+ example, using lambda expressions + KTable joined = left.leftJoin(right, foreignKeyExtractor, + (leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue /* ValueJoiner */ + ); + +Detailed behavior: + + * The join is _key-based_ , i.e. with the join predicate: + + foreignKeyExtractor.apply(leftRecord.value) == rightRecord.key + + * The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied `ValueJoiner` will be called to produce join output records. + +> * Input records with a `null` value are interpreted as _tombstones_ for the corresponding key, which indicate the deletion of the key from the table. Right-tombstones trigger the join, but left-tombstones don't: when an input tombstone is received, then an output tombstone is forwarded directly to the join result KTable if required (i.e. only if the corresponding key actually exists already in the join result KTable). +> * When joining versioned tables, out-of-order input records, i.e., those for which another record from the same table, with the same key and a larger timestamp, has already been processed, are ignored and do not trigger the join. + + * For each input record on the left side that does not have any match on the right side, the `ValueJoiner` will be called with `ValueJoiner#apply(leftRecord.value, null)`; this explains the row with timestamp=7 & 8 in the table below, which lists `(q,10,null) and (r,10,null)` in the LEFT JOIN column. + +See the semantics overview at the bottom of this section for a detailed description. + +**Semantics of table-table foreign-key joins:** The semantics of the table-table foreign-key INNER and LEFT JOIN variants are demonstrated below. The key is shown alongside the value for each record. Records are processed in incrementing offset order. The columns INNER JOIN and LEFT JOIN denote what is passed as arguments to the user-supplied [ValueJoiner](/%7B%7Bversion%7D%7D/javadoc/org/apache/kafka/streams/kstream/ValueJoiner.html) for the `join` and `leftJoin` methods, respectively, whenever a new input record is received on either side of the join. An empty table cell denotes that the `ValueJoiner` is not called at all. For the purpose of this example, `Function foreignKeyExtractor` simply uses the left-value as the output. + +Record Offset | Left KTable (K, extracted-FK) | Right KTable (FK, VR) | (INNER) JOIN | LEFT JOIN +---|---|---|---|--- +1 | (k,1) | (1,foo) | (k,1,foo) +| (k,1,foo) +2 | (k,2) | +| (k,null) | (k,2,null) + +3 | (k,3) +| | (k,null) | (k,3,null) + +4 | | (3,bar) +| (k,3,bar) +| (k,3,bar) + +5 | (k,null) +| | (k,null) +| (k,null,null) +6 | (k,1) | +| (k,1,foo) +| (k,1,foo) + +7 | (q,10) +| | +| (q,10,null) +8 | (r,10) | +| | (r,10,null) +9 | +| (10,baz) | (q,10,baz), (r,10,baz) | (q,10,baz), (r,10,baz) + +#### KStream-KTable Join + +KStream-KTable joins are always _non-windowed_ joins. They allow you to perform _table lookups_ against a KTable (changelog stream) upon receiving a new record from the KStream (record stream). An example use case would be to enrich a stream of user activities (KStream) with the latest user profile information (KTable). + +Join output records are effectively created as follows, leveraging the user-supplied `ValueJoiner`: + + + KeyValue leftRecord = ...; + KeyValue rightRecord = ...; + ValueJoiner joiner = ...; + + KeyValue joinOutputRecord = KeyValue.pair( + leftRecord.key, /* by definition, leftRecord.key == rightRecord.key */ + joiner.apply(leftRecord.value, rightRecord.value) + ); + +Transformation | Description +---|--- +**Inner Join** + + * (KStream, KTable) -> KStream + +| Performs an INNER JOIN of this stream with the table, effectively doing a table lookup. [(details)](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#join-org.apache.kafka.streams.kstream.KTable-org.apache.kafka.streams.kstream.ValueJoiner-) **Data must be co-partitioned** : The input data for both sides must be co-partitioned. **Causes data re-partitioning of the stream if and only if the stream was marked for re-partitioning.** Several variants of `join` exists, see the Javadocs for details. + + + KStream left = ...; + KTable right = ...; + + // Java 8+ example, using lambda expressions + KStream joined = left.join(right, + (leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue, /* ValueJoiner */ + Joined.keySerde(Serdes.String()) /* key */ + .withValueSerde(Serdes.Long()) /* left value */ + .withGracePeriod(Duration.ZERO) /* grace period */ + ); + + // Java 7 example + KStream joined = left.join(right, + new ValueJoiner() { + @Override + public String apply(Long leftValue, Double rightValue) { + return "left=" + leftValue + ", right=" + rightValue; + } + }, + Joined.keySerde(Serdes.String()) /* key */ + .withValueSerde(Serdes.Long()) /* left value */ + .withGracePeriod(Duration.ZERO) /* grace period */ + ); + +Detailed behavior: + + * The join is _key-based_ , i.e. with the join predicate `leftRecord.key == rightRecord.key`. + * The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied `ValueJoiner` will be called to produce join output records. + +> * Only input records for the left side (stream) trigger the join. Input records for the right side (table) update only the internal right-side join state. +> * Input records for the stream with a `null` key or a `null` value are ignored and do not trigger the join. +> * Input records for the table with a `null` value are interpreted as _tombstones_ for the corresponding key, which indicate the deletion of the key from the table. Tombstones do not trigger the join. + + * When the table is versioned, the table record to join with is determined by performing a timestamped lookup, i.e., the table record which is joined will be the latest-by-timestamp record with timestamp less than or equal to the stream record timestamp. If the stream record timestamp is older than the table's history retention, then the record is dropped. + * To use the grace period, the table needs to be versioned. This will cause the stream to buffer for the specified grace period before trying to find a matching record with the right timestamp in the table. The case where the grace period would be used for is if a record in the table has a timestamp less than or equal to the stream record timestamp but arrives after the stream record. If the table record arrives within the grace period the join will still occur. If the table record does not arrive before the grace period the join will continue as normal. + +See the semantics overview at the bottom of this section for a detailed description. +**Left Join** + + * (KStream, KTable) -> KStream + +| Performs a LEFT JOIN of this stream with the table, effectively doing a table lookup. [(details)](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#leftJoin-org.apache.kafka.streams.kstream.KTable-org.apache.kafka.streams.kstream.ValueJoiner-) **Data must be co-partitioned** : The input data for both sides must be co-partitioned. **Causes data re-partitioning of the stream if and only if the stream was marked for re-partitioning.** Several variants of `leftJoin` exists, see the Javadocs for details. + + + KStream left = ...; + KTable right = ...; + + // Java 8+ example, using lambda expressions + KStream joined = left.leftJoin(right, + (leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue, /* ValueJoiner */ + Joined.keySerde(Serdes.String()) /* key */ + .withValueSerde(Serdes.Long()) /* left value */ + .withGracePeriod(Duration.ZERO) /* grace period */ + ); + + // Java 7 example + KStream joined = left.leftJoin(right, + new ValueJoiner() { + @Override + public String apply(Long leftValue, Double rightValue) { + return "left=" + leftValue + ", right=" + rightValue; + } + }, + Joined.keySerde(Serdes.String()) /* key */ + .withValueSerde(Serdes.Long()) /* left value */ + .withGracePeriod(Duration.ZERO) /* grace period */ + ); + +Detailed behavior: + + * The join is _key-based_ , i.e. with the join predicate `leftRecord.key == rightRecord.key`. + * The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied `ValueJoiner` will be called to produce join output records. + +> * Only input records for the left side (stream) trigger the join. Input records for the right side (table) update only the internal right-side join state. +> * Input records for the stream with a `null` value are ignored and do not trigger the join. +> * Input records for the table with a `null` value are interpreted as _tombstones_ for the corresponding key, which indicate the deletion of the key from the table. Tombstones do not trigger the join. + + * For each input record on the left side that does not have any match on the right side, the `ValueJoiner` will be called with `ValueJoiner#apply(leftRecord.value, null)`; this explains the row with timestamp=3 in the table below, which lists `[A, null]` in the LEFT JOIN column. + * When the table is versioned, the table record to join with is determined by performing a timestamped lookup, i.e., the table record which is joined will be the latest-by-timestamp record with timestamp less than or equal to the stream record timestamp. If the stream record timestamp is older than the table's history retention, then the record that is joined will be `null`. + * To use the grace period, the table needs to be versioned. This will cause the stream to buffer for the specified grace period before trying to find a matching record with the right timestamp in the table. The case where the grace period would be used for is if a record in the table has a timestamp less than or equal to the stream record timestamp but arrives after the stream record. If the table record arrives within the grace period the join will still occur. If the table record does not arrive before the grace period the join will continue as normal. + +See the semantics overview at the bottom of this section for a detailed description. + +**Semantics of stream-table joins:** The semantics of the various stream-table join variants are explained below. To improve the readability of the table we assume that (1) all records have the same key (and thus we omit the key in the table) and that (2) all records are processed in timestamp order. The columns INNER JOIN and LEFT JOIN denote what is passed as arguments to the user-supplied [ValueJoiner](/37/javadoc/org/apache/kafka/streams/kstream/ValueJoiner.html) for the `join` and `leftJoin` methods, respectively, whenever a new input record is received on either side of the join. An empty table cell denotes that the `ValueJoiner` is not called at all. + +Timestamp | Left (KStream) | Right (KTable) | (INNER) JOIN | LEFT JOIN +---|---|---|---|--- +1 | null | | | +2 | | null | | +3 | A | | | [A, null] +4 | | a | | +5 | B | | [B, a] | [B, a] +6 | | b | | +7 | null | | | +8 | | null | | +9 | C | | | [C, null] +10 | | c | | +11 | | null | | +12 | null | | | +13 | | null | | +14 | | d | | +15 | D | | [D, d] | [D, d] + +#### KStream-GlobalKTable Join + +KStream-GlobalKTable joins are always _non-windowed_ joins. They allow you to perform _table lookups_ against a GlobalKTable (entire changelog stream) upon receiving a new record from the KStream (record stream). An example use case would be "star queries" or "star joins", where you would enrich a stream of user activities (KStream) with the latest user profile information (GlobalKTable) and further context information (further GlobalKTables). However, because GlobalKTables have no notion of time, a KStream-GlobalKTable join is not a temporal join, and there is no event-time synchronization between updates to a GlobalKTable and processing of KStream records. + +At a high-level, KStream-GlobalKTable joins are very similar to KStream-KTable joins. However, global tables provide you with much more flexibility at the some expense when compared to partitioned tables: + + * They do not require data co-partitioning. + * They allow for efficient "star joins"; i.e., joining a large-scale "facts" stream against "dimension" tables + * They allow for joining against foreign keys; i.e., you can lookup data in the table not just by the keys of records in the stream, but also by data in the record values. + * They make many use cases feasible where you must work on heavily skewed data and thus suffer from hot partitions. + * They are often more efficient than their partitioned KTable counterpart when you need to perform multiple joins in succession. + + + +Join output records are effectively created as follows, leveraging the user-supplied `ValueJoiner`: + + + KeyValue leftRecord = ...; + KeyValue rightRecord = ...; + ValueJoiner joiner = ...; + + KeyValue joinOutputRecord = KeyValue.pair( + leftRecord.key, /* by definition, leftRecord.key == rightRecord.key */ + joiner.apply(leftRecord.value, rightRecord.value) + ); + +Transformation | Description +---|--- +**Inner Join** + + * (KStream, GlobalKTable) -> KStream + +| Performs an INNER JOIN of this stream with the global table, effectively doing a table lookup. [(details)](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#join-org.apache.kafka.streams.kstream.GlobalKTable-org.apache.kafka.streams.kstream.KeyValueMapper-org.apache.kafka.streams.kstream.ValueJoiner-) The `GlobalKTable` is fully bootstrapped upon (re)start of a `KafkaStreams` instance, which means the table is fully populated with all the data in the underlying topic that is available at the time of the startup. The actual data processing begins only once the bootstrapping has completed. **Causes data re-partitioning of the stream if and only if the stream was marked for re-partitioning.** + + + KStream left = ...; + GlobalKTable right = ...; + + // Java 8+ example, using lambda expressions + KStream joined = left.join(right, + (leftKey, leftValue) -> leftKey.length(), /* derive a (potentially) new key by which to lookup against the table */ + (leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue /* ValueJoiner */ + ); + + // Java 7 example + KStream joined = left.join(right, + new KeyValueMapper() { /* derive a (potentially) new key by which to lookup against the table */ + @Override + public Integer apply(String key, Long value) { + return key.length(); + } + }, + new ValueJoiner() { + @Override + public String apply(Long leftValue, Double rightValue) { + return "left=" + leftValue + ", right=" + rightValue; + } + }); + +Detailed behavior: + + * The join is indirectly _key-based_ , i.e. with the join predicate `KeyValueMapper#apply(leftRecord.key, leftRecord.value) == rightRecord.key`. + * The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied `ValueJoiner` will be called to produce join output records. + +> * Only input records for the left side (stream) trigger the join. Input records for the right side (table) update only the internal right-side join state. +> * Input records for the stream with a `null` key or a `null` value are ignored and do not trigger the join. +> * Input records for the table with a `null` value are interpreted as _tombstones_ , which indicate the deletion of a record key from the table. Tombstones do not trigger the join. + + + +**Left Join** + + * (KStream, GlobalKTable) -> KStream + +| Performs a LEFT JOIN of this stream with the global table, effectively doing a table lookup. [(details)](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#leftJoin-org.apache.kafka.streams.kstream.GlobalKTable-org.apache.kafka.streams.kstream.KeyValueMapper-org.apache.kafka.streams.kstream.ValueJoiner-) The `GlobalKTable` is fully bootstrapped upon (re)start of a `KafkaStreams` instance, which means the table is fully populated with all the data in the underlying topic that is available at the time of the startup. The actual data processing begins only once the bootstrapping has completed. **Causes data re-partitioning of the stream if and only if the stream was marked for re-partitioning.** + + + KStream left = ...; + GlobalKTable right = ...; + + // Java 8+ example, using lambda expressions + KStream joined = left.leftJoin(right, + (leftKey, leftValue) -> leftKey.length(), /* derive a (potentially) new key by which to lookup against the table */ + (leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue /* ValueJoiner */ + ); + + // Java 7 example + KStream joined = left.leftJoin(right, + new KeyValueMapper() { /* derive a (potentially) new key by which to lookup against the table */ + @Override + public Integer apply(String key, Long value) { + return key.length(); + } + }, + new ValueJoiner() { + @Override + public String apply(Long leftValue, Double rightValue) { + return "left=" + leftValue + ", right=" + rightValue; + } + }); + +Detailed behavior: + + * The join is indirectly _key-based_ , i.e. with the join predicate `KeyValueMapper#apply(leftRecord.key, leftRecord.value) == rightRecord.key`. + * The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied `ValueJoiner` will be called to produce join output records. + +> * Only input records for the left side (stream) trigger the join. Input records for the right side (table) update only the internal right-side join state. +> * Input records for the stream with a `null` value are ignored and do not trigger the join. +> * Input records for the table with a `null` value are interpreted as _tombstones_ , which indicate the deletion of a record key from the table. Tombstones do not trigger the join. + + * For each input record on the left side that does not have any match on the right side, the `ValueJoiner` will be called with `ValueJoiner#apply(leftRecord.value, null)`. + + + +**Semantics of stream-global-table joins:** The join semantics are different to KStream-KTable joins because it's not a temporal join. Another difference is that, for KStream-GlobalKTable joins, the left input record is first "mapped" with a user-supplied `KeyValueMapper` into the table's keyspace prior to the table lookup. + +### Windowing + +Windowing lets you control how to group records that have the same key for stateful operations such as aggregations or joins into so-called windows. Windows are tracked per record key. + +**Note** + +A related operation is grouping, which groups all records that have the same key to ensure that data is properly partitioned ("keyed") for subsequent operations. Once grouped, windowing allows you to further sub-group the records of a key. + +For example, in join operations, a windowing state store is used to store all the records received so far within the defined window boundary. In aggregating operations, a windowing state store is used to store the latest aggregation results per window. Old records in the state store are purged after the specified [window retention period](../core-concepts.html#streams_concepts_windowing). Kafka Streams guarantees to keep a window for at least this specified time; the default value is one day and can be changed via `Materialized#withRetention()`. + +The DSL supports the following types of windows: + +Window name | Behavior | Short description +---|---|--- +Hopping time window | Time-based | Fixed-size, overlapping windows +Tumbling time window | Time-based | Fixed-size, non-overlapping, gap-less windows +Sliding time window | Time-based | Fixed-size, overlapping windows that work on differences between record timestamps +Session window | Session-based | Dynamically-sized, non-overlapping, data-driven windows + +#### Hopping time windows + +Hopping time windows are windows based on time intervals. They model fixed-sized, (possibly) overlapping windows. A hopping window is defined by two properties: the window's _size_ and its _advance interval_ (aka "hop"). The advance interval specifies by how much a window moves forward relative to the previous one. For example, you can configure a hopping window with a size 5 minutes and an advance interval of 1 minute. Since hopping windows can overlap - and in general they do - a data record may belong to more than one such windows. + +**Note** + +**Hopping windows vs. sliding windows:** Hopping windows are sometimes called "sliding windows" in other stream processing tools. Kafka Streams follows the terminology in academic literature, where the semantics of sliding windows are different to those of hopping windows. + +The following code defines a hopping window with a size of 5 minutes and an advance interval of 1 minute: + + + import java.time.Duration; + import org.apache.kafka.streams.kstream.TimeWindows; + + // A hopping time window with a size of 5 minutes and an advance interval of 1 minute. + // The window's name -- the string parameter -- is used to e.g. name the backing state store. + Duration windowSize = Duration.ofMinutes(5); + Duration advance = Duration.ofMinutes(1); + TimeWindows.ofSizeWithNoGrace(windowSize).advanceBy(advance); + +![](/37/images/streams-time-windows-hopping.png) + +This diagram shows windowing a stream of data records with hopping windows. In this diagram the time numbers represent minutes; e.g. t=5 means "at the five-minute mark". In reality, the unit of time in Kafka Streams is milliseconds, which means the time numbers would need to be multiplied with 60 * 1,000 to convert from minutes to milliseconds (e.g. t=5 would become t=300,000). + +Hopping time windows are _aligned to the epoch_ , with the lower interval bound being inclusive and the upper bound being exclusive. "Aligned to the epoch" means that the first window starts at timestamp zero. For example, hopping windows with a size of 5000ms and an advance interval ("hop") of 3000ms have predictable window boundaries `[0;5000),[3000;8000),...` -- and **not** `[1000;6000),[4000;9000),...` or even something "random" like `[1452;6452),[4452;9452),...`. + +Unlike non-windowed aggregates that we have seen previously, windowed aggregates return a _windowed KTable_ whose keys type is `Windowed`. This is to differentiate aggregate values with the same key from different windows. The corresponding window instance and the embedded key can be retrieved as `Windowed#window()` and `Windowed#key()`, respectively. + +#### Tumbling time windows + +Tumbling time windows are a special case of hopping time windows and, like the latter, are windows based on time intervals. They model fixed-size, non-overlapping, gap-less windows. A tumbling window is defined by a single property: the window's _size_. A tumbling window is a hopping window whose window size is equal to its advance interval. Since tumbling windows never overlap, a data record will belong to one and only one window. + +![](/37/images/streams-time-windows-tumbling.png) + +This diagram shows windowing a stream of data records with tumbling windows. Windows do not overlap because, by definition, the advance interval is identical to the window size. In this diagram the time numbers represent minutes; e.g. t=5 means "at the five-minute mark". In reality, the unit of time in Kafka Streams is milliseconds, which means the time numbers would need to be multiplied with 60 * 1,000 to convert from minutes to milliseconds (e.g. t=5 would become t=300,000). + +Tumbling time windows are _aligned to the epoch_ , with the lower interval bound being inclusive and the upper bound being exclusive. "Aligned to the epoch" means that the first window starts at timestamp zero. For example, tumbling windows with a size of 5000ms have predictable window boundaries `[0;5000),[5000;10000),...` -- and **not** `[1000;6000),[6000;11000),...` or even something "random" like `[1452;6452),[6452;11452),...`. + +The following code defines a tumbling window with a size of 5 minutes: + + + import java.time.Duration; + import org.apache.kafka.streams.kstream.TimeWindows; + + // A tumbling time window with a size of 5 minutes (and, by definition, an implicit + // advance interval of 5 minutes), and grace period of 1 minute. + Duration windowSize = Duration.ofMinutes(5); + Duration gracePeriod = Duration.ofMinutes(1); + TimeWindows.ofSizeAndGrace(windowSize, gracePeriod); + + // The above is equivalent to the following code: + TimeWindows.ofSizeAndGrace(windowSize, gracePeriod).advanceBy(windowSize); + +#### Sliding time windows + +Sliding windows are actually quite different from hopping and tumbling windows. In Kafka Streams, sliding windows are used for join operations, specified by using the `JoinWindows` class, and windowed aggregations, specified by using the `SlidingWindows` class. + +A sliding window models a fixed-size window that slides continuously over the time axis. In this model, two data records are said to be included in the same window if (in the case of symmetric windows) the difference of their timestamps is within the window size. As a sliding window moves along the time axis, records may fall into multiple snapshots of the sliding window, but each unique combination of records appears only in one sliding window snapshot. + +The following code defines a sliding window with a time difference of 10 minutes and a grace period of 30 minutes: + + + import org.apache.kafka.streams.kstream.SlidingWindows; + + // A sliding time window with a time difference of 10 minutes and grace period of 30 minutes + Duration timeDifference = Duration.ofMinutes(10); + Duration gracePeriod = Duration.ofMinutes(30); + SlidingWindows.ofTimeDifferenceAndGrace(timeDifference, gracePeriod); + +![](/37/images/streams-sliding-windows.png) + +This diagram shows windowing a stream of data records with sliding windows. The overlap of the sliding window snapshots varies depending on the record times. In this diagram, the time numbers represent milliseconds. For example, t=5 means "at the five millisecond mark". + +Sliding windows are aligned to the data record timestamps, not to the epoch. In contrast to hopping and tumbling windows, the lower and upper window time interval bounds of sliding windows are both inclusive. + +#### Session Windows + +Session windows are used to aggregate key-based events into so-called _sessions_ , the process of which is referred to as _sessionization_. Sessions represent a **period of activity** separated by a defined **gap of inactivity** (or "idleness"). Any events processed that fall within the inactivity gap of any existing sessions are merged into the existing sessions. If an event falls outside of the session gap, then a new session will be created. + +Session windows are different from the other window types in that: + + * all windows are tracked independently across keys - e.g. windows of different keys typically have different start and end times + * their window sizes sizes vary - even windows for the same key typically have different sizes + + + +The prime area of application for session windows is **user behavior analysis**. Session-based analyses can range from simple metrics (e.g. count of user visits on a news website or social platform) to more complex metrics (e.g. customer conversion funnel and event flows). + +The following code defines a session window with an inactivity gap of 5 minutes: + + + import java.time.Duration; + import org.apache.kafka.streams.kstream.SessionWindows; + + // A session window with an inactivity gap of 5 minutes. + SessionWindows.ofInactivityGapWithNoGrace(Duration.ofMinutes(5)); + +Given the previous session window example, here's what would happen on an input stream of six records. When the first three records arrive (upper part of in the diagram below), we'd have three sessions (see lower part) after having processed those records: two for the green record key, with one session starting and ending at the 0-minute mark (only due to the illustration it looks as if the session goes from 0 to 1), and another starting and ending at the 6-minute mark; and one session for the blue record key, starting and ending at the 2-minute mark. + +![](/37/images/streams-session-windows-01.png) + +Detected sessions after having received three input records: two records for the green record key at t=0 and t=6, and one record for the blue record key at t=2. In this diagram the time numbers represent minutes; e.g. t=5 means "at the five-minute mark". In reality, the unit of time in Kafka Streams is milliseconds, which means the time numbers would need to be multiplied with 60 * 1,000 to convert from minutes to milliseconds (e.g. t=5 would become t=300,000). + +If we then receive three additional records (including two out-of-order records), what would happen is that the two existing sessions for the green record key will be merged into a single session starting at time 0 and ending at time 6, consisting of a total of three records. The existing session for the blue record key will be extended to end at time 5, consisting of a total of two records. And, finally, there will be a new session for the blue key starting and ending at time 11. + +![](/37/images/streams-session-windows-02.png) + +Detected sessions after having received six input records. Note the two out-of-order data records at t=4 (green) and t=5 (blue), which lead to a merge of sessions and an extension of a session, respectively. + +#### Window Final Results + +In Kafka Streams, windowed computations update their results continuously. As new data arrives for a window, freshly computed results are emitted downstream. For many applications, this is ideal, since fresh results are always available. and Kafka Streams is designed to make programming continuous computations seamless. However, some applications need to take action **only** on the final result of a windowed computation. Common examples of this are sending alerts or delivering results to a system that doesn't support updates. + +Suppose that you have an hourly windowed count of events per user. If you want to send an alert when a user has _less than_ three events in an hour, you have a real challenge. All users would match this condition at first, until they accrue enough events, so you cannot simply send an alert when someone matches the condition; you have to wait until you know you won't see any more events for a particular window and _then_ send the alert. + +Kafka Streams offers a clean way to define this logic: after defining your windowed computation, you can suppress the intermediate results, emitting the final count for each user when the window is **closed**. + +For example: + + + KGroupedStream grouped = ...; + grouped + .windowedBy(TimeWindows.ofSizeAndGrace(Duration.ofHours(1), Duration.ofMinutes(10))) + .count() + .suppress(Suppressed.untilWindowCloses(unbounded())) + .filter((windowedUserId, count) -> count < 3) + .toStream() + .foreach((windowedUserId, count) -> sendAlert(windowedUserId.window(), windowedUserId.key(), count)); + +The key parts of this program are: + +`ofSizeAndGrace(Duration.ofHours(1), Duration.ofMinutes(10))` + The specified grace period of 10 minutes (i.e., the `Duration.ofMinutes(10)` argument) allows us to bound the lateness of events the window will accept. For example, the 09:00 to 10:00 window will accept out-of-order records until 10:10, at which point, the window is **closed**. +`.suppress(Suppressed.untilWindowCloses(...))` + This configures the suppression operator to emit nothing for a window until it closes, and then emit the final result. For example, if user `U` gets 10 events between 09:00 and 10:10, the `filter` downstream of the suppression will get no events for the windowed key `U@09:00-10:00` until 10:10, and then it will get exactly one with the value `10`. This is the final result of the windowed count. +`unbounded()` + This configures the buffer used for storing events until their windows close. Production code is able to put a cap on the amount of memory to use for the buffer, but this simple example creates a buffer with no upper bound. + +One thing to note is that suppression is just like any other Kafka Streams operator, so you can build a topology with two branches emerging from the `count`, one suppressed, and one not, or even multiple differently configured suppressions. This allows you to apply suppressions where they are needed and otherwise rely on the default continuous update behavior. + +For more detailed information, see the JavaDoc on the `Suppressed` config object and [KIP-328](https://cwiki.apache.org/confluence/x/sQU0BQ "KIP-328"). + +## Applying processors and transformers (Processor API integration) + +Beyond the aforementioned stateless and stateful transformations, you may also leverage the [Processor API](processor-api.html#streams-developer-guide-processor-api) from the DSL. There are a number of scenarios where this may be helpful: + + * **Customization:** You need to implement special, customized logic that is not or not yet available in the DSL. + * **Combining ease-of-use with full flexibility where it 's needed:** Even though you generally prefer to use the expressiveness of the DSL, there are certain steps in your processing that require more flexibility and tinkering than the DSL provides. For example, only the Processor API provides access to a record's metadata such as its topic, partition, and offset information. However, you don't want to switch completely to the Processor API just because of that. + * **Migrating from other tools:** You are migrating from other stream processing technologies that provide an imperative API, and migrating some of your legacy code to the Processor API was faster and/or easier than to migrate completely to the DSL right away. + +Transformation | Description +---|--- +**Process** + + * KStream -> void + +| **Terminal operation.** Applies a `Processor` to each record. `process()` allows you to leverage the [Processor API](processor-api.html#streams-developer-guide-processor-api) from the DSL. ([details](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#process-org.apache.kafka.streams.processor.ProcessorSupplier-java.lang.String...-)) This is essentially equivalent to adding the `Processor` via `Topology#addProcessor()` to your [processor topology](../core-concepts.html#streams_topology). An example is available in the [javadocs](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#process-org.apache.kafka.streams.processor.ProcessorSupplier-java.lang.String...-). +**Transform** + + * KStream -> KStream + +| Applies a `Transformer` to each record. `transform()` allows you to leverage the [Processor API](processor-api.html#streams-developer-guide-processor-api) from the DSL. ([details](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#transform-org.apache.kafka.streams.kstream.TransformerSupplier-java.lang.String...-)) Each input record is transformed into zero, one, or more output records (similar to the stateless `flatMap`). The `Transformer` must return `null` for zero output. You can modify the record's key and value, including their types. **Marks the stream for data re-partitioning:** Applying a grouping or a join after `transform` will result in re-partitioning of the records. If possible use `transformValues` instead, which will not cause data re-partitioning. `transform` is essentially equivalent to adding the `Transformer` via `Topology#addProcessor()` to your [processor topology](../core-concepts.html#streams_topology). An example is available in the [javadocs](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#transform-org.apache.kafka.streams.kstream.TransformerSupplier-java.lang.String...-). +**Transform (values only)** + + * KStream -> KStream + * KTable -> KTable + +| Applies a `ValueTransformer` to each record, while retaining the key of the original record. `transformValues()` allows you to leverage the [Processor API](processor-api.html#streams-developer-guide-processor-api) from the DSL. ([details](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#transformValues-org.apache.kafka.streams.kstream.ValueTransformerSupplier-java.lang.String...-)) Each input record is transformed into exactly one output record (zero output records or multiple output records are not possible). The `ValueTransformer` may return `null` as the new value for a record. `transformValues` is preferable to `transform` because it will not cause data re-partitioning. `transformValues` is essentially equivalent to adding the `ValueTransformer` via `Topology#addProcessor()` to your [processor topology](../core-concepts.html#streams_topology). An example is available in the [javadocs](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#transformValues-org.apache.kafka.streams.kstream.ValueTransformerSupplier-java.lang.String...-). + +**CAUTION:** If you are using "merge repartition topics" optimization, it is not recommended to use `KStream#processValues` to avoid compatibility issues for future upgrades to newer versions of Kafka Streams. For more details, see the [migration guide](/40/streams/developer-guide/dsl-api.htm#transformers-removal-and-migration-to-processors) in the Kafka Streams 4.0 docs. + +The following example shows how to leverage, via the `KStream#process()` method, a custom `Processor` that sends an email notification whenever a page view count reaches a predefined threshold. + +First, we need to implement a custom stream processor, `PopularPageEmailAlert`, that implements the `Processor` interface: + + + // A processor that sends an alert message about a popular page to a configurable email address + public class PopularPageEmailAlert implements Processor { + + private final String emailAddress; + private ProcessorContext context; + + public PopularPageEmailAlert(String emailAddress) { + this.emailAddress = emailAddress; + } + + @Override + public void init(ProcessorContext context) { + this.context = context; + + // Here you would perform any additional initializations such as setting up an email client. + } + + @Override + void process(Record record) { + // Here you would format and send the alert email. + // + // In this specific example, you would be able to include + // information about the page's ID and its view count + } + + @Override + void close() { + // Any code for clean up would go here, for example tearing down the email client and anything + // else you created in the init() method + // This processor instance will not be used again after this call. + } + + } + +**Tip** + +Even though we do not demonstrate it in this example, a stream processor can access any available state stores by calling `ProcessorContext#getStateStore()`. State stores are only available if they have been connected to the processor, or if they are global stores. While global stores do not need to be connected explicitly, they only allow for read-only access. There are two ways to connect state stores to a processor: + + * By passing the name of a store that has already been added via `Topology#addStateStore()` to the corresponding `KStream#process()` method call. + * Implementing `ConnectedStoreProvider#stores()` on the `ProcessorSupplier` passed to `KStream#process()`. In this case there is no need to call `StreamsBuilder#addStateStore()` beforehand, the store will be automatically added for you. You can also implement `ConnectedStoreProvider#stores()` on the `Value*` or `*WithKey` supplier variants, or `TransformerSupplier` or any of its variants. + + + +Then we can leverage the `PopularPageEmailAlert` processor in the DSL via `KStream#process`. + + + KStream pageViews = ...; + + // Send an email notification when the view count of a page reaches one thousand. + pageViews.groupByKey() + .count() + .filter((PageId pageId, Long viewCount) -> viewCount == 1000) + // PopularPageEmailAlert is your custom processor that implements the + // `Processor` interface, see further down below. + .process(() -> new PopularPageEmailAlert("alerts@yourcompany.com")); + +Naming Operators in a Streams DSL application Kafka Streams allows you to [name processors](dsl-topology-naming.html) created via the Streams DSL + +# Controlling KTable emit rate + +A KTable is logically a continuously updated table. These updates make their way to downstream operators whenever new data is available, ensuring that the whole computation is as fresh as possible. Logically speaking, most programs describe a series of transformations, and the update rate is not a factor in the program behavior. In these cases, the rate of update is more of a performance concern. Operators are able to optimize both the network traffic (to the Kafka brokers) and the disk traffic (to the local state stores) by adjusting commit interval and batch size configurations. + +However, some applications need to take other actions, such as calling out to external systems, and therefore need to exercise some control over the rate of invocations, for example of `KStream#foreach`. + +Rather than achieving this as a side-effect of the [KTable record cache](memory-mgmt.html#streams-developer-guide-memory-management-record-cache), you can directly impose a rate limit via the `KTable#suppress` operator. + +For example: + + + KGroupedTable groupedTable = ...; + groupedTable + .count() + .suppress(untilTimeLimit(ofMinutes(5), maxBytes(1_000_000L).emitEarlyWhenFull())) + .toStream() + .foreach((key, count) -> updateCountsDatabase(key, count)); + +This configuration ensures that `updateCountsDatabase` gets events for each `key` no more than once every 5 minutes. Note that the latest state for each key has to be buffered in memory for that 5-minute period. You have the option to control the maximum amount of memory to use for this buffer (in this case, 1MB). There is also an option to impose a limit in terms of number of records (or to leave both limits unspecified). + +Additionally, it is possible to choose what happens if the buffer fills up. This example takes a relaxed approach and just emits the oldest records before their 5-minute time limit to bring the buffer back down to size. Alternatively, you can choose to stop processing and shut the application down. This may seem extreme, but it gives you a guarantee that the 5-minute time limit will be absolutely enforced. After the application shuts down, you could allocate more memory for the buffer and resume processing. Emitting early is preferable for most applications. + +For more detailed information, see the JavaDoc on the `Suppressed` config object and [KIP-328](https://cwiki.apache.org/confluence/x/sQU0BQ "KIP-328"). + +# Using timestamp-based semantics for table processors + +By default, tables in Kafka Streams use offset-based semantics. When multiple records arrive for the same key, the one with the largest record offset is considered the latest record for the key, and is the record that appears in aggregation and join results computed on the table. This is true even in the event of [out-of-order data](/37/streams/core-concepts.html#streams_out_of_ordering). The record with the largest offset is considered to be the latest record for the key, even if this record does not have the largest timestamp. + +An alternative to offset-based semantics is timestamp-based semantics. With timestamp-based semantics, the record with the largest timestamp is considered the latest record, even if there is another record with a larger offset (and smaller timestamp). If there is no out-of-order data (per key), then offset-based semantics and timestamp-based semantics are equivalent; the difference only appears when there is out-of-order data. + +Starting with Kafka Streams 3.5, Kafka Streams supports timestamp-based semantics through the use of [versioned state stores](/37/streams/developer-guide/processor-api.html#versioned-state-stores). When a table is materialized with a versioned state store, it is a versioned table and will result in different processor semantics in the presence of out-of-order data. + + * When performing a stream-table join, stream-side records will join with the latest-by-timestamp table record which has a timestamp less than or equal to the stream record's timestamp. This is in contrast to joining a stream to an unversioned table, in which case the latest-by-offset table record will be joined, even if the stream-side record is out-of-order and has a lower timestamp. + * Aggregations computed on the table will include the latest-by-timestamp record for each key, instead of the latest-by-offset record. Out-of-order updates (per key) will not trigger a new aggregation result. This is true for `count` and `reduce` operations as well, in addition to `aggregate` operations. + * Table joins will use the latest-by-timestamp record for each key, instead of the latest-by-offset record. Out-of-order updates (per key) will not trigger a new join result. This is true for both primary-key table-table joins and also foreign-key table-table joins. If a versioned table is joined with an unversioned table, the result will be the join of the latest-by-timestamp record from the versioned table with the latest-by-offset record from the unversioned table. + * Table filter operations will no longer suppress consecutive tombstones, so users may observe more `null` records downstream of the filter than compared to when filtering an unversioned table. This is done in order to preserve a complete version history downstream, in the event of out-of-order data. + * `suppress` operations are not allowed on versioned tables, as this would collapse the version history and lead to undefined behavior. + + + +Once a table is materialized with a versioned store, downstream tables are also considered versioned until any of the following occurs: + + * A downstream table is explicitly materialized, either with an unversioned store supplier or with no store supplier (all stores are unversioned by default, including the default store supplier) + * Any stateful transformation occurs, including aggregations and joins + * A table is converted to a stream and back. + + + +The results of certain processors should not be materialized with versioned stores, as these processors do not produce a complete older version history, and therefore materialization as a versioned table would lead to unpredictable results: + + * Aggregate processors, for both table and stream aggregations. This includes `aggregate`, `count` and `reduce` operations. + * Table-table join processors, including both primary-key and foreign-key joins. + + + +For more on versioned stores and how to start using them in your application, see [here](/37/streams/developer-guide/processor-api.html#versioned-state-stores). + +# Writing streams back to Kafka + +Any streams and tables may be (continuously) written back to a Kafka topic. As we will describe in more detail below, the output data might be re-partitioned on its way to Kafka, depending on the situation. + +Writing to Kafka | Description +---|--- +**To** + + * KStream -> void + +| **Terminal operation.** Write the records to Kafka topic(s). ([KStream details](/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#to\(java.lang.String\))) When to provide serdes explicitly: + + * If you do not specify Serdes explicitly, the default Serdes from the [configuration](config-streams.html#streams-developer-guide-configuration) are used. + * You **must specify Serdes explicitly** via the `Produced` class if the key and/or value types of the `KStream` do not match the configured default Serdes. + * See [Data Types and Serialization](datatypes.html#streams-developer-guide-serdes) for information about configuring default Serdes, available Serdes, and implementing your own custom Serdes. + +A variant of `to` exists that enables you to specify how the data is produced by using a `Produced` instance to specify, for example, a `StreamPartitioner` that gives you control over how output records are distributed across the partitions of the output topic. Another variant of `to` exists that enables you to dynamically choose which topic to send to for each record via a `TopicNameExtractor` instance. + + + KStream stream = ...; + + // Write the stream to the output topic, using the configured default key + // and value serdes. + stream.to("my-stream-output-topic"); + + // Write the stream to the output topic, using explicit key and value serdes, + // (thus overriding the defaults in the config properties). + stream.to("my-stream-output-topic", Produced.with(Serdes.String(), Serdes.Long()); + +**Causes data re-partitioning if any of the following conditions is true:** + + 1. If the output topic has a different number of partitions than the stream/table. + 2. If the `KStream` was marked for re-partitioning. + 3. If you provide a custom `StreamPartitioner` to explicitly control how to distribute the output records across the partitions of the output topic. + 4. If the key of an output record is `null`. + + + +**Note** + +**When you want to write to systems other than Kafka:** Besides writing the data back to Kafka, you can also apply a custom processor as a stream sink at the end of the processing to, for example, write to external databases. First, doing so is not a recommended pattern - we strongly suggest to use the [Kafka Connect API](../../connect/index.html#kafka-connect) instead. However, if you do use such a sink processor, please be aware that it is now your responsibility to guarantee message delivery semantics when talking to such external systems (e.g., to retry on delivery failure or to prevent message duplication). + +# Testing a Streams application + +Kafka Streams comes with a `test-utils` module to help you test your application [here](testing.html). + +# Kafka Streams DSL for Scala + +The Kafka Streams DSL Java APIs are based on the Builder design pattern, which allows users to incrementally build the target functionality using lower level compositional fluent APIs. These APIs can be called from Scala, but there are several issues: + + 1. **Additional type annotations** \- The Java APIs use Java generics in a way that are not fully compatible with the type inferencer of the Scala compiler. Hence the user has to add type annotations to the Scala code, which seems rather non-idiomatic in Scala. + 2. **Verbosity** \- In some cases the Java APIs appear too verbose compared to idiomatic Scala. + 3. **Type Unsafety** \- The Java APIs offer some options where the compile time type safety is sometimes subverted and can result in runtime errors. This stems from the fact that the Serdes defined as part of config are not type checked during compile time. Hence any missing Serdes can result in runtime errors. + + + +The Kafka Streams DSL for Scala library is a wrapper over the existing Java APIs for Kafka Streams DSL that addresses the concerns raised above. It does not attempt to provide idiomatic Scala APIs that one would implement in a Scala library developed from scratch. The intention is to make the Java APIs more usable in Scala through better type inferencing, enhanced expressiveness, and lesser boilerplates. + +The library wraps Java Stream DSL APIs in Scala thereby providing: + + 1. Better type inference in Scala. + 2. Less boilerplate in application code. + 3. The usual builder-style composition that developers get with the original Java API. + 4. Implicit serializers and de-serializers leading to better abstraction and less verbosity. + 5. Better type safety during compile time. + + + +All functionality provided by Kafka Streams DSL for Scala are under the root package name of `org.apache.kafka.streams.scala`. + +Many of the public facing types from the Java API are wrapped. The following Scala abstractions are available to the user: + + * `org.apache.kafka.streams.scala.StreamsBuilder` + * `org.apache.kafka.streams.scala.kstream.KStream` + * `org.apache.kafka.streams.scala.kstream.KTable` + * `org.apache.kafka.streams.scala.kstream.KGroupedStream` + * `org.apache.kafka.streams.scala.kstream.KGroupedTable` + * `org.apache.kafka.streams.scala.kstream.SessionWindowedKStream` + * `org.apache.kafka.streams.scala.kstream.TimeWindowedKStream` + + + +The library also has several utility abstractions and modules that the user needs to use for proper semantics. + + * `org.apache.kafka.streams.scala.ImplicitConversions`: Module that brings into scope the implicit conversions between the Scala and Java classes. + * `org.apache.kafka.streams.scala.serialization.Serdes`: Module that contains all primitive Serdes that can be imported as implicits and a helper to create custom Serdes. + + + +The library is cross-built with Scala 2.12 and 2.13. To reference the library compiled against Scala 2.13 include the following in your maven `pom.xml` add the following: + + + + org.apache.kafka + kafka-streams-scala_2.13 + 3.7.2 + + +To use the library compiled against Scala 2.12 replace the `artifactId` with `kafka-streams-scala_2.12`. + +When using SBT then you can reference the correct library using the following: + + + libraryDependencies += "org.apache.kafka" %% "kafka-streams-scala" % "3.7.2" + +## Sample Usage + +The library works by wrapping the original Java abstractions of Kafka Streams within a Scala wrapper object and then using implicit conversions between them. All the Scala abstractions are named identically as the corresponding Java abstraction, but they reside in a different package of the library e.g. the Scala class `org.apache.kafka.streams.scala.StreamsBuilder` is a wrapper around `org.apache.kafka.streams.StreamsBuilder`, `org.apache.kafka.streams.scala.kstream.KStream` is a wrapper around `org.apache.kafka.streams.kstream.KStream`, and so on. + +Here's an example of the classic WordCount program that uses the Scala `StreamsBuilder` that builds an instance of `KStream` which is a wrapper around Java `KStream`. Then we reify to a table and get a `KTable`, which, again is a wrapper around Java `KTable`. + +The net result is that the following code is structured just like using the Java API, but with Scala and with far fewer type annotations compared to using the Java API directly from Scala. The difference in type annotation usage is more obvious when given an example. Below is an example WordCount implementation that will be used to demonstrate the differences between the Scala and Java API. + + + import java.time.Duration + import java.util.Properties + + import org.apache.kafka.streams.kstream.Materialized + import org.apache.kafka.streams.scala.ImplicitConversions._ + import org.apache.kafka.streams.scala._ + import org.apache.kafka.streams.scala.kstream._ + import org.apache.kafka.streams.{KafkaStreams, StreamsConfig} + + object WordCountApplication extends App { + import Serdes._ + + val props: Properties = { + val p = new Properties() + p.put(StreamsConfig.APPLICATION_ID_CONFIG, "wordcount-application") + p.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka-broker1:9092") + p + } + + val builder: StreamsBuilder = new StreamsBuilder + val textLines: KStream[String, String] = builder.stream[String, String]("TextLinesTopic") + val wordCounts: KTable[String, Long] = textLines + .flatMapValues(textLine => textLine.toLowerCase.split("\W+")) + .groupBy((_, word) => word) + .count(Materialized.as("counts-store")) + wordCounts.toStream.to("WordsWithCountsTopic") + + val streams: KafkaStreams = new KafkaStreams(builder.build(), props) + streams.start() + + sys.ShutdownHookThread { + streams.close(Duration.ofSeconds(10)) + } + } + +In the above code snippet, we don't have to provide any Serdes, `Grouped`, `Produced`, `Consumed` or `Joined` explicitly. They will also not be dependent on any Serdes specified in the config. **In fact all Serdes specified in the config will be ignored by the Scala APIs**. All Serdes and `Grouped`, `Produced`, `Consumed` or `Joined` will be handled through implicit Serdes as discussed later in the Implicit Serdes section. The complete independence from configuration based Serdes is what makes this library completely typesafe. Any missing instances of Serdes, `Grouped`, `Produced`, `Consumed` or `Joined` will be flagged as a compile time error. + +## Implicit Serdes + +One of the common complaints of Scala users with the Java API has been the repetitive usage of the Serdes in API invocations. Many of the APIs need to take the Serdes through abstractions like `Grouped`, `Produced`, `Repartitioned`, `Consumed` or `Joined`. And the user has to supply them every time through the with function of these classes. + +The library uses the power of [Scala implicit parameters](https://docs.scala-lang.org/tour/implicit-parameters.html) to alleviate this concern. As a user you can provide implicit Serdes or implicit values of `Grouped`, `Produced`, `Repartitioned`, `Consumed` or `Joined` once and make your code less verbose. In fact you can just have the implicit Serdes in scope and the library will make the instances of `Grouped`, `Produced`, `Consumed` or `Joined` available in scope. + +The library also bundles all implicit Serdes of the commonly used primitive types in a Scala module - so just import the module vals and have all Serdes in scope. A similar strategy of modular implicits can be adopted for any user-defined Serdes as well (User-defined Serdes are discussed in the next section). + +Here's an example: + + + // DefaultSerdes brings into scope implicit Serdes (mostly for primitives) + // that will set up all Grouped, Produced, Consumed and Joined instances. + // So all APIs below that accept Grouped, Produced, Consumed or Joined will + // get these instances automatically + import Serdes._ + + val builder = new StreamsBuilder() + + val userClicksStream: KStream[String, Long] = builder.stream(userClicksTopic) + + val userRegionsTable: KTable[String, String] = builder.table(userRegionsTopic) + + // The following code fragment does not have a single instance of Grouped, + // Produced, Consumed or Joined supplied explicitly. + // All of them are taken care of by the implicit Serdes imported by DefaultSerdes + val clicksPerRegion: KTable[String, Long] = + userClicksStream + .leftJoin(userRegionsTable)((clicks, region) => (if (region == null) "UNKNOWN" else region, clicks)) + .map((_, regionWithClicks) => regionWithClicks) + .groupByKey + .reduce(_ + _) + + clicksPerRegion.toStream.to(outputTopic) + +Quite a few things are going on in the above code snippet that may warrant a few lines of elaboration: + + 1. The code snippet does not depend on any config defined Serdes. In fact any Serdes defined as part of the config will be ignored. + 2. All Serdes are picked up from the implicits in scope. And `import Serdes._` brings all necessary Serdes in scope. + 3. This is an example of compile time type safety that we don't have in the Java APIs. + 4. The code looks less verbose and more focused towards the actual transformation that it does on the data stream. + + + +## User-Defined Serdes + +When the default primitive Serdes are not enough and we need to define custom Serdes, the usage is exactly the same as above. Just define the implicit Serdes and start building the stream transformation. Here's an example with `AvroSerde`: + + + // domain object as a case class + case class UserClicks(clicks: Long) + + // An implicit Serde implementation for the values we want to + // serialize as avro + implicit val userClicksSerde: Serde[UserClicks] = new AvroSerde + + // Primitive Serdes + import Serdes._ + + // And then business as usual .. + + val userClicksStream: KStream[String, UserClicks] = builder.stream(userClicksTopic) + + val userRegionsTable: KTable[String, String] = builder.table(userRegionsTopic) + + // Compute the total per region by summing the individual click counts per region. + val clicksPerRegion: KTable[String, Long] = + userClicksStream + + // Join the stream against the table. + .leftJoin(userRegionsTable)((clicks, region) => (if (region == null) "UNKNOWN" else region, clicks.clicks)) + + // Change the stream from -> to -> + .map((_, regionWithClicks) => regionWithClicks) + + // Compute the total per region by summing the individual click counts per region. + .groupByKey + .reduce(_ + _) + + // Write the (continuously updating) results to the output topic. + clicksPerRegion.toStream.to(outputTopic) + +A complete example of user-defined Serdes can be found in a test class within the library. + + * [Documentation](/documentation) + * [Kafka Streams](/streams) + * [Developer Guide](/streams/developer-guide/) + + diff --git a/docs/streams/developer-guide/dsl-topology-naming.html b/docs/streams/developer-guide/dsl-topology-naming.html deleted file mode 100644 index 72e0c2887a12e..0000000000000 --- a/docs/streams/developer-guide/dsl-topology-naming.html +++ /dev/null @@ -1,350 +0,0 @@ - - - - - - - - - - - - - - - diff --git a/docs/streams/developer-guide/dsl-topology-naming.md b/docs/streams/developer-guide/dsl-topology-naming.md new file mode 100644 index 0000000000000..c5ed970a4df41 --- /dev/null +++ b/docs/streams/developer-guide/dsl-topology-naming.md @@ -0,0 +1,186 @@ +--- +title: Naming Operators in a Streams DSL application +description: +weight: 5 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +# Developer Guide for Kafka Streams + +# Naming Operators in a Kafka Streams DSL Application + +You now can give names to processors when using the Kafka Streams DSL. In the PAPI there are `Processors` and `State Stores` and you are required to explicitly name each one. + +At the DSL layer, there are operators. A single DSL operator may compile down to multiple `Processors` and `State Stores`, and if required `repartition topics`. But with the Kafka Streams DSL, all these names are generated for you. There is a relationship between the generated processor name state store names (hence changelog topic names) and repartition topic names. Note, that the names of state stores and changelog/repartition topics are "stateful" while processor names are "stateless". + +This distinction of stateful vs. stateless names has important implications when updating your topology. While the internal naming makes creating a topology with the DSL much more straightforward, there are a couple of trade-offs. The first trade-off is what we could consider a readability issue. The other more severe trade-off is the shifting of names due to the relationship between the DSL operator and the generated `Processors`, `State Stores` changelog topics and repartition topics. + +# Readability Issues + +By saying there is a readability trade-off, we are referring to viewing a description of the topology. When you render the string description of your topology via the `Topology#describe()` method, you can see what the processor is, but you don't have any context for its business purpose. For example, consider the following simple topology: + + + + KStream stream = builder.stream("input"); + stream.filter((k,v) -> !v.equals("invalid_txn")) + .mapValues((v) -> v.substring(0,5)) + .to("output"); + +Running `Topology#describe()` yields this string: + + + Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input]) + --> KSTREAM-FILTER-0000000001 + Processor: KSTREAM-FILTER-0000000001 (stores: []) + --> KSTREAM-MAPVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-MAPVALUES-0000000002 (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-FILTER-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: output) + <-- KSTREAM-MAPVALUES-0000000002 + +From this report, you can see what the different operators are, but what is the broader context here? For example, consider `KSTREAM-FILTER-0000000001`, we can see that it's a filter operation, which means that records are dropped that don't match the given predicate. But what is the meaning of the predicate? Additionally, you can see the topic names of the source and sink nodes, but what if the topics aren't named in a meaningful way? Then you're left to guess the business purpose behind these topics. + +Also notice the numbering here: the source node is suffixed with `0000000000` indicating it's the first processor in the topology. The filter is suffixed with `0000000001`, indicating it's the second processor in the topology. In Kafka Streams, there are now overloaded methods for both `KStream` and `KTable` that accept a new parameter `Named`. By using the `Named` class DSL users can provide meaningful names to the processors in their topology. + +Now let's take a look at your topology with all the processors named: + + + KStream stream = + builder.stream("input", Consumed.as("Customer_transactions_input_topic")); + stream.filter((k,v) -> !v.equals("invalid_txn"), Named.as("filter_out_invalid_txns")) + .mapValues((v) -> v.substring(0,5), Named.as("Map_values_to_first_6_characters")) + .to("output", Produced.as("Mapped_transactions_output_topic")); + + + Topologies: + Sub-topology: 0 + Source: Customer_transactions_input_topic (topics: [input]) + --> filter_out_invalid_txns + Processor: filter_out_invalid_txns (stores: []) + --> Map_values_to_first_6_characters + <-- Customer_transactions_input_topic + Processor: Map_values_to_first_6_characters (stores: []) + --> Mapped_transactions_output_topic + <-- filter_out_invalid_txns + Sink: Mapped_transactions_output_topic (topic: output) + <-- Map_values_to_first_6_characters + +Now you can look at the topology description and easily understand what role each processor plays in the topology. But there's another reason for naming your processor nodes when you have stateful operators that remain between restarts of your Kafka Streams applications, state stores, changelog topics, and repartition topics. + +# Changing Names + +Generated names are numbered where they are built in the topology. The name generation strategy is `KSTREAM|KTABLE->operator name<->number suffix<`. The number is a globally incrementing number that represents the operator's order in the topology. The generated number is prefixed with a varying number of "0"s to create a string that is consistently 10 characters long. This means that if you add/remove or shift the order of operations, the position of the processor shifts, which shifts the name of the processor. Since **most** processors exist in memory only, this name shifting presents no issue for many topologies. But the name shifting does have implications for topologies with stateful operators or repartition topics. Here's a different topology with some state: + + + KStream stream = builder.stream("input"); + stream.groupByKey() + .count() + .toStream() + .to("output"); + +This topology description yields the following: + + + Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input]) + --> KSTREAM-AGGREGATE-0000000002 + Processor: KSTREAM-AGGREGATE-0000000002 (stores: [KSTREAM-AGGREGATE-STATE-STORE-0000000001]) + --> KTABLE-TOSTREAM-0000000003 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TOSTREAM-0000000003 (stores: []) + --> KSTREAM-SINK-0000000004 + <-- KSTREAM-AGGREGATE-0000000002 + Sink: KSTREAM-SINK-0000000004 (topic: output) + <-- KTABLE-TOSTREAM-0000000003 + +You can see from the topology description above that the state store is named `KSTREAM-AGGREGATE-STATE-STORE-0000000002`. Here's what happens when you add a filter to keep some of the records out of the aggregation: + + + KStream stream = builder.stream("input"); + stream.filter((k,v)-> v !=null && v.length() >= 6 ) + .groupByKey() + .count() + .toStream() + .to("output"); + +And the corresponding topology: + + + Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input]) + --> KSTREAM-FILTER-0000000001 + Processor: KSTREAM-FILTER-0000000001 (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [KSTREAM-AGGREGATE-STATE-STORE-0000000002]) + --> KTABLE-TOSTREAM-0000000004 + <-- KSTREAM-FILTER-0000000001 + Processor: KTABLE-TOSTREAM-0000000004 (stores: []) + --> KSTREAM-SINK-0000000005 + <-- KSTREAM-AGGREGATE-0000000003 + Sink: KSTREAM-SINK-0000000005 (topic: output) + <-- KTABLE-TOSTREAM-0000000004 + +Notice that since you've added an operation _before_ the `count` operation, the state store (and the changelog topic) names have changed. This name change means you can't do a rolling re-deployment of your updated topology. Also, you must use the [Streams Reset Tool](/37/streams/developer-guide/app-reset-tool) to re-calculate the aggregations, because the changelog topic has changed on start-up and the new changelog topic contains no data. Fortunately, there's an easy solution to remedy this situation. Give the state store a user-defined name instead of relying on the generated one, so you don't have to worry about topology changes shifting the name of the state store. You've had the ability to name repartition topics with the `Joined`, `StreamJoined`, and`Grouped` classes, and name state store and changelog topics with `Materialized`. But it's worth reiterating the importance of naming these DSL topology operations again. Here's how your DSL code looks now giving a specific name to your state store: + + + KStream stream = builder.stream("input"); + stream.filter((k, v) -> v != null && v.length() >= 6) + .groupByKey() + .count(Materialized.as("Purchase_count_store")) + .toStream() + .to("output"); + +And here's the topology + + + Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input]) + --> KSTREAM-FILTER-0000000001 + Processor: KSTREAM-FILTER-0000000001 (stores: []) + --> KSTREAM-AGGREGATE-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-AGGREGATE-0000000002 (stores: [Purchase_count_store]) + --> KTABLE-TOSTREAM-0000000003 + <-- KSTREAM-FILTER-0000000001 + Processor: KTABLE-TOSTREAM-0000000003 (stores: []) + --> KSTREAM-SINK-0000000004 + <-- KSTREAM-AGGREGATE-0000000002 + Sink: KSTREAM-SINK-0000000004 (topic: output) + <-- KTABLE-TOSTREAM-0000000003 + +Now, even though you've added processors before your state store, the store name and its changelog topic names don't change. This makes your topology more robust and resilient to changes made by adding or removing processors. + +# Conclusion + +It's a good practice to name your processing nodes when using the DSL, and it's even more important to do this when you have "stateful" processors your application such as repartition topics and state stores (and the accompanying changelog topics). + +Here are a couple of points to remember when naming your DSL topology: + + 1. If you have an _existing topology_ and you _haven't_ named your state stores (and changelog topics) and repartition topics, we recommended that you do so. But this will be a topology breaking change, so you'll need to shut down all application instances, make the changes, and run the [Streams Reset Tool](/37/streams/developer-guide/app-reset-tool). Although this may be inconvenient at first, it's worth the effort to protect your application from unexpected errors due to topology changes. + 2. If you have a _new topology_ , make sure you name the persistent parts of your topology: state stores (changelog topics) and repartition topics. This way, when you deploy your application, you're protected from topology changes that otherwise would break your Kafka Streams application. If you don't want to add names to stateless processors at first, that's fine as you can always go back and add the names later. + +Here's a quick reference on naming the critical parts of your Kafka Streams application to prevent topology name changes from breaking your application: Operation| Naming Class +---|--- +Aggregation repartition topics| Grouped +KStream-KStream Join repartition topics| StreamJoined +KStream-KTable Join repartition topic| Joined +KStream-KStream Join state stores| StreamJoined +State Stores (for aggregations and KTable-KTable joins)| Materialized +Stream/Table non-stateful operations| Named + + * [Documentation](/documentation) + * [Kafka Streams](/streams) + * [Developer Guide](/streams/developer-guide/) + + diff --git a/docs/streams/developer-guide/index.html b/docs/streams/developer-guide/index.html deleted file mode 100644 index 19f638ece0daf..0000000000000 --- a/docs/streams/developer-guide/index.html +++ /dev/null @@ -1,106 +0,0 @@ - - - - - - - - -
- - -
- - diff --git a/docs/streams/developer-guide/interactive-queries.html b/docs/streams/developer-guide/interactive-queries.html deleted file mode 100644 index 9160dd8ddf40a..0000000000000 --- a/docs/streams/developer-guide/interactive-queries.html +++ /dev/null @@ -1,503 +0,0 @@ - - - - - - - - - - - diff --git a/docs/streams/developer-guide/interactive-queries.md b/docs/streams/developer-guide/interactive-queries.md new file mode 100644 index 0000000000000..9ba2c0153b095 --- /dev/null +++ b/docs/streams/developer-guide/interactive-queries.md @@ -0,0 +1,418 @@ +--- +title: Interactive Queries +description: +weight: 8 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +Interactive queries allow you to leverage the state of your application from outside your application. The Kafka Streams enables your applications to be queryable. + + + + +The full state of your application is typically [split across many distributed instances of your application](../architecture.html#streams_architecture_state), and across many state stores that are managed locally by these application instances. + +![](/37/images/streams-interactive-queries-03.png) + +There are local and remote components to interactively querying the state of your application. + +Local state + An application instance can query the locally managed portion of the state and directly query its own local state stores. You can use the corresponding local data in other parts of your application code, as long as it doesn't require calling the Kafka Streams API. Querying state stores is always read-only to guarantee that the underlying state stores will never be mutated out-of-band (e.g., you cannot add new entries). State stores should only be mutated by the corresponding processor topology and the input data it operates on. For more information, see Querying local state stores for an app instance. +Remote state + + +To query the full state of your application, you must connect the various fragments of the state, including: + + * query local state stores + * discover all running instances of your application in the network and their state stores + * communicate with these instances over the network (e.g., an RPC layer) + + + +Connecting these fragments enables communication between instances of the same app and communication from other applications for interactive queries. For more information, see Querying remote state stores for the entire app. + +Kafka Streams natively provides all of the required functionality for interactively querying the state of your application, except if you want to expose the full state of your application via interactive queries. To allow application instances to communicate over the network, you must add a Remote Procedure Call (RPC) layer to your application (e.g., REST API). + +This table shows the Kafka Streams native communication support for various procedures. + +Procedure | Application instance | Entire application +---|---|--- +Query local state stores of an app instance | Supported | Supported +Make an app instance discoverable to others | Supported | Supported +Discover all running app instances and their state stores | Supported | Supported +Communicate with app instances over the network (RPC) | Supported | Not supported (you must configure) + +# Querying local state stores for an app instance + +A Kafka Streams application typically runs on multiple instances. The state that is locally available on any given instance is only a subset of the [application's entire state](../architecture.html#streams-architecture-state). Querying the local stores on an instance will only return data locally available on that particular instance. + +The method `KafkaStreams#store(...)` finds an application instance's local state stores by name and type. Note that interactive queries are not supported for [versioned state stores](processor-api.html#streams-developer-guide-state-store-versioned) at this time. + +![](/37/images/streams-interactive-queries-api-01.png) + +Every application instance can directly query any of its local state stores. + +The _name_ of a state store is defined when you create the store. You can create the store explicitly by using the Processor API or implicitly by using stateful operations in the DSL. + +The _type_ of a state store is defined by `QueryableStoreType`. You can access the built-in types via the class `QueryableStoreTypes`. Kafka Streams currently has two built-in types: + + * A key-value store `QueryableStoreTypes#keyValueStore()`, see Querying local key-value stores. + * A window store `QueryableStoreTypes#windowStore()`, see Querying local window stores. + + + +You can also implement your own QueryableStoreType as described in section Querying local custom state stores. + +**Note** + +Kafka Streams materializes one state store per stream partition. This means your application will potentially manage many underlying state stores. The API enables you to query all of the underlying stores without having to know which partition the data is in. + +## Querying local key-value stores + +To query a local key-value store, you must first create a topology with a key-value store. This example creates a key-value store named "CountsKeyValueStore". This store will hold the latest count for any word that is found on the topic "word-count-input". + + + Properties props = ...; + StreamsBuilder builder = ...; + KStream textLines = ...; + + // Define the processing topology (here: WordCount) + KGroupedStream groupedByWord = textLines + .flatMapValues(value -> Arrays.asList(value.toLowerCase().split("\W+"))) + .groupBy((key, word) -> word, Grouped.with(stringSerde, stringSerde)); + + // Create a key-value store named "CountsKeyValueStore" for the all-time word counts + groupedByWord.count(Materialized.as("CountsKeyValueStore")); + + // Start an instance of the topology + KafkaStreams streams = new KafkaStreams(builder, props); + streams.start(); + +After the application has started, you can get access to "CountsKeyValueStore" and then query it via the [ReadOnlyKeyValueStore](https://github.com/apache/kafka/blob/3.7/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java) API: + + + // Get the key-value store CountsKeyValueStore + ReadOnlyKeyValueStore keyValueStore = + streams.store("CountsKeyValueStore", QueryableStoreTypes.keyValueStore()); + + // Get value by key + System.out.println("count for hello:" + keyValueStore.get("hello")); + + // Get the values for a range of keys available in this application instance + KeyValueIterator range = keyValueStore.range("all", "streams"); + while (range.hasNext()) { + KeyValue next = range.next(); + System.out.println("count for " + next.key + ": " + next.value); + } + + // Get the values for all of the keys available in this application instance + KeyValueIterator range = keyValueStore.all(); + while (range.hasNext()) { + KeyValue next = range.next(); + System.out.println("count for " + next.key + ": " + next.value); + } + +You can also materialize the results of stateless operators by using the overloaded methods that take a `queryableStoreName` as shown in the example below: + + + StreamsBuilder builder = ...; + KTable regionCounts = ...; + + // materialize the result of filtering corresponding to odd numbers + // the "queryableStoreName" can be subsequently queried. + KTable oddCounts = numberLines.filter((region, count) -> (count % 2 != 0), + Materialized.as("queryableStoreName")); + + // do not materialize the result of filtering corresponding to even numbers + // this means that these results will not be materialized and cannot be queried. + KTable oddCounts = numberLines.filter((region, count) -> (count % 2 == 0)); + +## Querying local window stores + +A window store will potentially have many results for any given key because the key can be present in multiple windows. However, there is only one result per window for a given key. + +To query a local window store, you must first create a topology with a window store. This example creates a window store named "CountsWindowStore" that contains the counts for words in 1-minute windows. + + + StreamsBuilder builder = ...; + KStream textLines = ...; + + // Define the processing topology (here: WordCount) + KGroupedStream groupedByWord = textLines + .flatMapValues(value -> Arrays.asList(value.toLowerCase().split("\W+"))) + .groupBy((key, word) -> word, Grouped.with(stringSerde, stringSerde)); + + // Create a window state store named "CountsWindowStore" that contains the word counts for every minute + groupedByWord.windowedBy(TimeWindows.ofSizeWithNoGrace(Duration.ofSeconds(60))) + .count(Materialized.as("CountsWindowStore")); + +After the application has started, you can get access to "CountsWindowStore" and then query it via the [ReadOnlyWindowStore](https://github.com/apache/kafka/blob/3.7/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java) API: + + + // Get the window store named "CountsWindowStore" + ReadOnlyWindowStore windowStore = + streams.store("CountsWindowStore", QueryableStoreTypes.windowStore()); + + // Fetch values for the key "world" for all of the windows available in this application instance. + // To get *all* available windows we fetch windows from the beginning of time until now. + Instant timeFrom = Instant.ofEpochMilli(0); // beginning of time = oldest available + Instant timeTo = Instant.now(); // now (in processing-time) + WindowStoreIterator iterator = windowStore.fetch("world", timeFrom, timeTo); + while (iterator.hasNext()) { + KeyValue next = iterator.next(); + long windowTimestamp = next.key; + System.out.println("Count of 'world' @ time " + windowTimestamp + " is " + next.value); + } + +## Querying local custom state stores + +**Note** + +Only the [Processor API](processor-api.html#streams-developer-guide-processor-api) supports custom state stores. + +Before querying the custom state stores you must implement these interfaces: + + * Your custom state store must implement `StateStore`. + * You must have an interface to represent the operations available on the store. + * You must provide an implementation of `StoreBuilder` for creating instances of your store. + * It is recommended that you provide an interface that restricts access to read-only operations. This prevents users of this API from mutating the state of your running Kafka Streams application out-of-band. + + + +The class/interface hierarchy for your custom store might look something like: + + + public class MyCustomStore implements StateStore, MyWriteableCustomStore { + // implementation of the actual store + } + + // Read-write interface for MyCustomStore + public interface MyWriteableCustomStore extends MyReadableCustomStore { + void write(K Key, V value); + } + + // Read-only interface for MyCustomStore + public interface MyReadableCustomStore { + V read(K key); + } + + public class MyCustomStoreBuilder implements StoreBuilder { + // implementation of the supplier for MyCustomStore + } + +To make this store queryable you must: + + * Provide an implementation of [QueryableStoreType](https://github.com/apache/kafka/blob/3.7/streams/src/main/java/org/apache/kafka/streams/state/QueryableStoreType.java). + * Provide a wrapper class that has access to all of the underlying instances of the store and is used for querying. + + + +Here is how to implement `QueryableStoreType`: + + + public class MyCustomStoreType implements QueryableStoreType> { + + // Only accept StateStores that are of type MyCustomStore + public boolean accepts(final StateStore stateStore) { + return stateStore instanceOf MyCustomStore; + } + + public MyReadableCustomStore create(final StateStoreProvider storeProvider, final String storeName) { + return new MyCustomStoreTypeWrapper(storeProvider, storeName, this); + } + + } + +A wrapper class is required because each instance of a Kafka Streams application may run multiple stream tasks and manage multiple local instances of a particular state store. The wrapper class hides this complexity and lets you query a "logical" state store by name without having to know about all of the underlying local instances of that state store. + +When implementing your wrapper class you must use the [StateStoreProvider](https://github.com/apache/kafka/blob/3.7/streams/src/main/java/org/apache/kafka/streams/state/internals/StateStoreProvider.java) interface to get access to the underlying instances of your store. `StateStoreProvider#stores(String storeName, QueryableStoreType queryableStoreType)` returns a `List` of state stores with the given storeName and of the type as defined by `queryableStoreType`. + +Here is an example implementation of the wrapper follows (Java 8+): + + + // We strongly recommended implementing a read-only interface + // to restrict usage of the store to safe read operations! + public class MyCustomStoreTypeWrapper implements MyReadableCustomStore { + + private final QueryableStoreType> customStoreType; + private final String storeName; + private final StateStoreProvider provider; + + public CustomStoreTypeWrapper(final StateStoreProvider provider, + final String storeName, + final QueryableStoreType> customStoreType) { + + // ... assign fields ... + } + + // Implement a safe read method + @Override + public V read(final K key) { + // Get all the stores with storeName and of customStoreType + final List> stores = provider.getStores(storeName, customStoreType); + // Try and find the value for the given key + final Optional value = stores.stream().filter(store -> store.read(key) != null).findFirst(); + // Return the value if it exists + return value.orElse(null); + } + + } + +You can now find and query your custom store: + + + Topology topology = ...; + ProcessorSupplier processorSuppler = ...; + + // Create CustomStoreSupplier for store name the-custom-store + MyCustomStoreBuilder customStoreBuilder = new MyCustomStoreBuilder("the-custom-store") //...; + // Add the source topic + topology.addSource("input", "inputTopic"); + // Add a custom processor that reads from the source topic + topology.addProcessor("the-processor", processorSupplier, "input"); + // Connect your custom state store to the custom processor above + topology.addStateStore(customStoreBuilder, "the-processor"); + + KafkaStreams streams = new KafkaStreams(topology, config); + streams.start(); + + // Get access to the custom store + MyReadableCustomStore store = streams.store("the-custom-store", new MyCustomStoreType()); + // Query the store + String value = store.read("key"); + +# Querying remote state stores for the entire app + +To query remote states for the entire app, you must expose the application's full state to other applications, including applications that are running on different machines. + +For example, you have a Kafka Streams application that processes user events in a multi-player video game, and you want to retrieve the latest status of each user directly and display it in a mobile app. Here are the required steps to make the full state of your application queryable: + + 1. Add an RPC layer to your application so that the instances of your application can be interacted with via the network (e.g., a REST API, Thrift, a custom protocol, and so on). The instances must respond to interactive queries. You can follow the reference examples provided to get started. + 2. Expose the RPC endpoints of your application's instances via the `application.server` configuration setting of Kafka Streams. Because RPC endpoints must be unique within a network, each instance has its own value for this configuration setting. This makes an application instance discoverable by other instances. + 3. In the RPC layer, discover remote application instances and their state stores and query locally available state stores to make the full state of your application queryable. The remote application instances can forward queries to other app instances if a particular instance lacks the local data to respond to a query. The locally available state stores can directly respond to queries. + + + +![](/37/images/streams-interactive-queries-api-02.png) + +Discover any running instances of the same application as well as the respective RPC endpoints they expose for interactive queries + +## Adding an RPC layer to your application + +There are many ways to add an RPC layer. The only requirements are that the RPC layer is embedded within the Kafka Streams application and that it exposes an endpoint that other application instances and applications can connect to. + +## Exposing the RPC endpoints of your application + +To enable remote state store discovery in a distributed Kafka Streams application, you must set the [configuration property](config-streams.html#streams-developer-guide-required-configs) in the config properties. The `application.server` property defines a unique `host:port` pair that points to the RPC endpoint of the respective instance of a Kafka Streams application. The value of this configuration property will vary across the instances of your application. When this property is set, Kafka Streams will keep track of the RPC endpoint information for every instance of an application, its state stores, and assigned stream partitions through instances of [StreamsMetadata](/37/javadoc/org/apache/kafka/streams/state/StreamsMetadata.html). + +**Tip** + +Consider leveraging the exposed RPC endpoints of your application for further functionality, such as piggybacking additional inter-application communication that goes beyond interactive queries. + +This example shows how to configure and run a Kafka Streams application that supports the discovery of its state stores. + + + Properties props = new Properties(); + // Set the unique RPC endpoint of this application instance through which it + // can be interactively queried. In a real application, the value would most + // probably not be hardcoded but derived dynamically. + String rpcEndpoint = "host1:4460"; + props.put(StreamsConfig.APPLICATION_SERVER_CONFIG, rpcEndpoint); + // ... further settings may follow here ... + + StreamsBuilder builder = new StreamsBuilder(); + + KStream textLines = builder.stream(stringSerde, stringSerde, "word-count-input"); + + final KGroupedStream groupedByWord = textLines + .flatMapValues(value -> Arrays.asList(value.toLowerCase().split("\W+"))) + .groupBy((key, word) -> word, Grouped.with(stringSerde, stringSerde)); + + // This call to `count()` creates a state store named "word-count". + // The state store is discoverable and can be queried interactively. + groupedByWord.count(Materialized.as("word-count")); + + // Start an instance of the topology + KafkaStreams streams = new KafkaStreams(builder, props); + streams.start(); + + // Then, create and start the actual RPC service for remote access to this + // application instance's local state stores. + // + // This service should be started on the same host and port as defined above by + // the property `StreamsConfig.APPLICATION_SERVER_CONFIG`. The example below is + // fictitious, but we provide end-to-end demo applications (such as KafkaMusicExample) + // that showcase how to implement such a service to get you started. + MyRPCService rpcService = ...; + rpcService.listenAt(rpcEndpoint); + +## Discovering and accessing application instances and their local state stores + +The following methods return [StreamsMetadata](/37/javadoc/org/apache/kafka/streams/state/StreamsMetadata.html) objects, which provide meta-information about application instances such as their RPC endpoint and locally available state stores. + + * `KafkaStreams#allMetadata()`: find all instances of this application + * `KafkaStreams#allMetadataForStore(String storeName)`: find those applications instances that manage local instances of the state store "storeName" + * `KafkaStreams#metadataForKey(String storeName, K key, Serializer keySerializer)`: using the default stream partitioning strategy, find the one application instance that holds the data for the given key in the given state store + * `KafkaStreams#metadataForKey(String storeName, K key, StreamPartitioner partitioner)`: using `partitioner`, find the one application instance that holds the data for the given key in the given state store + + + +Attention + +If `application.server` is not configured for an application instance, then the above methods will not find any [StreamsMetadata](/37/javadoc/org/apache/kafka/streams/state/StreamsMetadata.html) for it. + +For example, we can now find the `StreamsMetadata` for the state store named "word-count" that we defined in the code example shown in the previous section: + + + KafkaStreams streams = ...; + // Find all the locations of local instances of the state store named "word-count" + Collection wordCountHosts = streams.allMetadataForStore("word-count"); + + // For illustrative purposes, we assume using an HTTP client to talk to remote app instances. + HttpClient http = ...; + + // Get the word count for word (aka key) 'alice': Approach 1 + // + // We first find the one app instance that manages the count for 'alice' in its local state stores. + StreamsMetadata metadata = streams.metadataForKey("word-count", "alice", Serdes.String().serializer()); + // Then, we query only that single app instance for the latest count of 'alice'. + // Note: The RPC URL shown below is fictitious and only serves to illustrate the idea. Ultimately, + // the URL (or, in general, the method of communication) will depend on the RPC layer you opted to + // implement. Again, we provide end-to-end demo applications (such as KafkaMusicExample) that showcase + // how to implement such an RPC layer. + Long result = http.getLong("http://" + metadata.host() + ":" + metadata.port() + "/word-count/alice"); + + // Get the word count for word (aka key) 'alice': Approach 2 + // + // Alternatively, we could also choose (say) a brute-force approach where we query every app instance + // until we find the one that happens to know about 'alice'. + Optional result = streams.allMetadataForStore("word-count") + .stream() + .map(streamsMetadata -> { + // Construct the (fictituous) full endpoint URL to query the current remote application instance + String url = "http://" + streamsMetadata.host() + ":" + streamsMetadata.port() + "/word-count/alice"; + // Read and return the count for 'alice', if any. + return http.getLong(url); + }) + .filter(s -> s != null) + .findFirst(); + +At this point the full state of the application is interactively queryable: + + * You can discover the running instances of the application and the state stores they manage locally. + * Through the RPC layer that was added to the application, you can communicate with these application instances over the network and query them for locally available state. + * The application instances are able to serve such queries because they can directly query their own local state stores and respond via the RPC layer. + * Collectively, this allows us to query the full state of the entire application. + + + +To see an end-to-end application with interactive queries, review the demo applications. + + * [Documentation](/documentation) + * [Kafka Streams](/streams) + * [Developer Guide](/streams/developer-guide/) + + diff --git a/docs/streams/developer-guide/manage-topics.html b/docs/streams/developer-guide/manage-topics.html deleted file mode 100644 index b41055ad20c53..0000000000000 --- a/docs/streams/developer-guide/manage-topics.html +++ /dev/null @@ -1,137 +0,0 @@ - - - - - - - - - - - diff --git a/docs/streams/developer-guide/manage-topics.md b/docs/streams/developer-guide/manage-topics.md new file mode 100644 index 0000000000000..a16c6fce028a7 --- /dev/null +++ b/docs/streams/developer-guide/manage-topics.md @@ -0,0 +1,61 @@ +--- +title: Managing Streams Application Topics +description: +weight: 11 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +A Kafka Streams application continuously reads from Kafka topics, processes the read data, and then writes the processing results back into Kafka topics. The application may also auto-create other Kafka topics in the Kafka brokers, for example state store changelogs topics. This section describes the differences these topic types and how to manage the topics and your applications. + +Kafka Streams distinguishes between user topics and internal topics. + +# User topics + +User topics exist externally to an application and are read from or written to by the application, including: + +Input topics + Topics that are specified via source processors in the application's topology; e.g. via `StreamsBuilder#stream()`, `StreamsBuilder#table()` and `Topology#addSource()`. +Output topics + Topics that are specified via sink processors in the application's topology; e.g. via `KStream#to()`, `KTable.to()` and `Topology#addSink()`. +Intermediate topics + Topics that are both input and output topics of the application's topology. + +User topics must be created and manually managed ahead of time (e.g., via the [topic tools](../../kafka/post-deployment.html#kafka-operations-admin)). If user topics are shared among multiple applications for reading and writing, the application users must coordinate topic management. If user topics are centrally managed, then application users then would not need to manage topics themselves but simply obtain access to them. + +Note + +You should not use the auto-create topic feature on the brokers to create user topics, because: + + * Auto-creation of topics may be disabled in your Kafka cluster. + * Auto-creation automatically applies the default topic settings such as the replicaton factor. These default settings might not be what you want for certain output topics (e.g., `auto.create.topics.enable=true` in the [Kafka broker configuration](http://kafka.apache.org/0100/documentation.html#brokerconfigs)). + + + +# Internal topics + +Internal topics are used internally by the Kafka Streams application while executing, for example the changelog topics for state stores. These topics are created by the application and are only used by that stream application. + +If security is enabled on the Kafka brokers, you must grant the underlying clients admin permissions so that they can create internal topics set. For more information, see [Streams Security](security.html#streams-developer-guide-security). + +Note + +The internal topics follow the naming convention `--`, but this convention is not guaranteed for future releases. + +The following settings apply to the default configuration for internal topics: + + * For all internal topics, `message.timestamp.type` is set to `CreateTime`. + * For internal repartition topics, the compaction policy is `delete` and the retention time is `-1` (infinite). + * For internal changelog topics for key-value stores, the compaction policy is `compact`. + * For internal changelog topics for windowed key-value stores, the compaction policy is `delete,compact`. The retention time is set to 24 hours plus your setting for the windowed store. + * For internal changelog topics for versioned state stores, the cleanup policy is `compact`, and `min.compaction.lag.ms` is set to 24 hours plus the store's historyRetentionMs` value. + + + + * [Documentation](/documentation) + * [Kafka Streams](/streams) + * [Developer Guide](/streams/developer-guide/) + + diff --git a/docs/streams/developer-guide/memory-mgmt.html b/docs/streams/developer-guide/memory-mgmt.html deleted file mode 100644 index 7f5bc09cef5b9..0000000000000 --- a/docs/streams/developer-guide/memory-mgmt.html +++ /dev/null @@ -1,279 +0,0 @@ - - - - - - - - - - - diff --git a/docs/streams/developer-guide/memory-mgmt.md b/docs/streams/developer-guide/memory-mgmt.md new file mode 100644 index 0000000000000..1e132f0222bec --- /dev/null +++ b/docs/streams/developer-guide/memory-mgmt.md @@ -0,0 +1,185 @@ +--- +title: Memory Management +description: +weight: 9 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +You can specify the total memory (RAM) size used for internal caching and compacting of records. This caching happens before the records are written to state stores or forwarded downstream to other nodes. + +The record caches are implemented slightly different in the DSL and Processor API. + + + + +# Record caches in the DSL + +You can specify the total memory (RAM) size of the record cache for an instance of the processing topology. It is leveraged by the following `KTable` instances: + + * Source `KTable`: `KTable` instances that are created via `StreamsBuilder#table()` or `StreamsBuilder#globalTable()`. + * Aggregation `KTable`: instances of `KTable` that are created as a result of [aggregations](dsl-api.html#streams-developer-guide-dsl-aggregating). + + + +For such `KTable` instances, the record cache is used for: + + * Internal caching and compacting of output records before they are written by the underlying stateful [processor node](../core-concepts#streams_processor_node) to its internal state stores. + * Internal caching and compacting of output records before they are forwarded from the underlying stateful [processor node](../core-concepts#streams_processor_node) to any of its downstream processor nodes. + + + +Use the following example to understand the behaviors with and without record caching. In this example, the input is a `KStream` with the records `: , , , `. The focus in this example is on the records with key == `A`. + + * An [aggregation](dsl-api.html#streams-developer-guide-dsl-aggregating) computes the sum of record values, grouped by key, for the input and returns a `KTable`. + +> * **Without caching** : a sequence of output records is emitted for key `A` that represent changes in the resulting aggregation table. The parentheses (`()`) denote changes, the left number is the new aggregate value and the right number is the old aggregate value: `, , `. +> * **With caching** : a single output record is emitted for key `A` that would likely be compacted in the cache, leading to a single output record of ``. This record is written to the aggregation's internal state store and forwarded to any downstream operations. + + + + +The cache size is specified through the `cache.max.bytes.buffering` parameter, which is a global setting per processing topology: + + + // Enable record cache of size 10 MB. + Properties props = new Properties(); + props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10 * 1024 * 1024L); + +This parameter controls the number of bytes allocated for caching. Specifically, for a processor topology instance with `T` threads and `C` bytes allocated for caching, each thread will have an even `C/T` bytes to construct its own cache and use as it sees fit among its tasks. This means that there are as many caches as there are threads, but no sharing of caches across threads happens. + +The basic API for the cache is made of `put()` and `get()` calls. Records are evicted using a simple LRU scheme after the cache size is reached. The first time a keyed record `R1 = ` finishes processing at a node, it is marked as dirty in the cache. Any other keyed record `R2 = ` with the same key `K1` that is processed on that node during that time will overwrite ``, this is referred to as "being compacted". This has the same effect as [Kafka's log compaction](https://kafka.apache.org/documentation.html#compaction), but happens earlier, while the records are still in memory, and within your client-side application, rather than on the server-side (i.e. the Kafka broker). After flushing, `R2` is forwarded to the next processing node and then written to the local state store. + +The semantics of caching is that data is flushed to the state store and forwarded to the next downstream processor node whenever the earliest of `commit.interval.ms` or `cache.max.bytes.buffering` (cache pressure) hits. Both `commit.interval.ms` and `cache.max.bytes.buffering` are global parameters. As such, it is not possible to specify different parameters for individual nodes. + +Here are example settings for both parameters based on desired scenarios. + + * To turn off caching the cache size can be set to zero: + + // Disable record cache + Properties props = new Properties(); + props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + + * To enable caching but still have an upper bound on how long records will be cached, you can set the commit interval. In this example, it is set to 1000 milliseconds: + + Properties props = new Properties(); + // Enable record cache of size 10 MB. + props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10 * 1024 * 1024L); + // Set commit interval to 1 second. + props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000); + + + + +The effect of these two configurations is described in the figure below. The records are shown using 4 keys: blue, red, yellow, and green. Assume the cache has space for only 3 keys. + + * When the cache is disabled (a), all of the input records will be output. + + * When the cache is enabled (b): + +> * Most records are output at the end of commit intervals (e.g., at `t1` a single blue record is output, which is the final over-write of the blue key up to that time). +> * Some records are output because of cache pressure (i.e. before the end of a commit interval). For example, see the red record before `t2`. With smaller cache sizes we expect cache pressure to be the primary factor that dictates when records are output. With large cache sizes, the commit interval will be the primary factor. +> * The total number of records output has been reduced from 15 to 8. + + + + +![](/37/images/streams-cache-and-commit-interval.png) + +# Record caches in the Processor API + +You can specify the total memory (RAM) size of the record cache for an instance of the processing topology. It is used for internal caching and compacting of output records before they are written from a stateful processor node to its state stores. + +The record cache in the Processor API does not cache or compact any output records that are being forwarded downstream. This means that all downstream processor nodes can see all records, whereas the state stores see a reduced number of records. This does not impact correctness of the system, but is a performance optimization for the state stores. For example, with the Processor API you can store a record in a state store while forwarding a different value downstream. + +Following from the example first shown in section [State Stores](processor-api.html#streams-developer-guide-state-store), to disable caching, you can add the `withCachingDisabled` call (note that caches are enabled by default, however there is an explicit `withCachingEnabled` call). + + + StoreBuilder countStoreBuilder = + Stores.keyValueStoreBuilder( + Stores.persistentKeyValueStore("Counts"), + Serdes.String(), + Serdes.Long()) + .withCachingEnabled(); + +Record caches are not supported for [versioned state stores](processor-api.html#streams-developer-guide-state-store-versioned). + +# RocksDB + +Each instance of RocksDB allocates off-heap memory for a block cache, index and filter blocks, and memtable (write buffer). Critical configs (for RocksDB version 4.1.0) include `block_cache_size`, `write_buffer_size` and `max_write_buffer_number`. These can be specified through the `rocksdb.config.setter` configuration. + +Also, we recommend changing RocksDB's default memory allocator, because the default allocator may lead to increased memory consumption. To change the memory allocator to `jemalloc`, you need to set the environment variable `LD_PRELOAD`before you start your Kafka Streams application: + + + # example: install jemalloc (on Debian) + $ apt install -y libjemalloc-dev + # set LD_PRELOAD before you start your Kafka Streams application + $ export LD_PRELOAD="/usr/lib/x86_64-linux-gnu/libjemalloc.so” + + +As of 2.3.0 the memory usage across all instances can be bounded, limiting the total off-heap memory of your Kafka Streams application. To do so you must configure RocksDB to cache the index and filter blocks in the block cache, limit the memtable memory through a shared [WriteBufferManager](https://github.com/facebook/rocksdb/wiki/Write-Buffer-Manager) and count its memory against the block cache, and then pass the same Cache object to each instance. See [RocksDB Memory Usage](https://github.com/facebook/rocksdb/wiki/Memory-usage-in-RocksDB) for details. An example RocksDBConfigSetter implementing this is shown below: + + + public static class BoundedMemoryRocksDBConfig implements RocksDBConfigSetter { + + private static org.rocksdb.Cache cache = new org.rocksdb.LRUCache(TOTAL_OFF_HEAP_MEMORY, -1, false, INDEX_FILTER_BLOCK_RATIO);1 + private static org.rocksdb.WriteBufferManager writeBufferManager = new org.rocksdb.WriteBufferManager(TOTAL_MEMTABLE_MEMORY, cache); + + @Override + public void setConfig(final String storeName, final Options options, final Map configs) { + + BlockBasedTableConfig tableConfig = (BlockBasedTableConfig) options.tableFormatConfig(); + + // These three options in combination will limit the memory used by RocksDB to the size passed to the block cache (TOTAL_OFF_HEAP_MEMORY) + tableConfig.setBlockCache(cache); + tableConfig.setCacheIndexAndFilterBlocks(true); + options.setWriteBufferManager(writeBufferManager); + + // These options are recommended to be set when bounding the total memory + tableConfig.setCacheIndexAndFilterBlocksWithHighPriority(true);2 + tableConfig.setPinTopLevelIndexAndFilter(true); + tableConfig.setBlockSize(BLOCK_SIZE);3 + options.setMaxWriteBufferNumber(N_MEMTABLES); + options.setWriteBufferSize(MEMTABLE_SIZE); + + options.setTableFormatConfig(tableConfig); + } + + @Override + public void close(final String storeName, final Options options) { + // Cache and WriteBufferManager should not be closed here, as the same objects are shared by every store instance. + } + } + +1\. INDEX_FILTER_BLOCK_RATIO can be used to set a fraction of the block cache to set aside for "high priority" (aka index and filter) blocks, preventing them from being evicted by data blocks. The boolean parameter in the cache constructor lets you control whether the cache should enforce a strict memory limit by failing the read or iteration in the rare cases where it might go larger than its capacity. See the full signature of the LRUCache constructor [here](https://github.com/facebook/rocksdb/blob/master/java/src/main/java/org/rocksdb/LRUCache.java#L72). +2\. This must be set in order for INDEX_FILTER_BLOCK_RATIO to take effect (see footnote 1) as described in the [RocksDB docs](https://github.com/facebook/rocksdb/wiki/Block-Cache#caching-index-and-filter-blocks) +3\. You may want to modify the default [block size](https://github.com/apache/kafka/blob/2.3/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java#L79) per these instructions from the [RocksDB docs](https://github.com/facebook/rocksdb/wiki/Memory-usage-in-RocksDB#indexes-and-filter-blocks). A larger block size means index blocks will be smaller, but the cached data blocks may contain more cold data that would otherwise be evicted. + + +Note: +While we recommend setting at least the above configs, the specific options that yield the best performance are workload dependent and you should consider experimenting with these to determine the best choices for your specific use case. Keep in mind that the optimal configs for one app may not apply to one with a different topology or input topic. In addition to the recommended configs above, you may want to consider using partitioned index filters as described by the [RocksDB docs](https://github.com/facebook/rocksdb/wiki/Partitioned-Index-Filters). + +# Other memory usage + +There are other modules inside Apache Kafka that allocate memory during runtime. They include the following: + + * Producer buffering, managed by the producer config `buffer.memory`. + * Consumer buffering, currently not strictly managed, but can be indirectly controlled by fetch size, i.e., `fetch.max.bytes` and `fetch.max.wait.ms`. + * Both producer and consumer also have separate TCP send / receive buffers that are not counted as the buffering memory. These are controlled by the `send.buffer.bytes` / `receive.buffer.bytes` configs. + * Deserialized objects buffering: after `consumer.poll()` returns records, they will be deserialized to extract timestamp and buffered in the streams space. Currently this is only indirectly controlled by `buffered.records.per.partition`. + + + +**Tip** + +**Iterators should be closed explicitly to release resources:** Store iterators (e.g., `KeyValueIterator` and `WindowStoreIterator`) must be closed explicitly upon completeness to release resources such as open file handlers and in-memory read buffers, or use try-with-resources statement (available since JDK7) for this Closeable class. + +Otherwise, stream application's memory usage keeps increasing when running until it hits an OOM. + + * [Documentation](/documentation) + * [Kafka Streams](/streams) + * [Developer Guide](/streams/developer-guide/) + + diff --git a/docs/streams/developer-guide/processor-api.html b/docs/streams/developer-guide/processor-api.html deleted file mode 100644 index ccb03ce7b50c1..0000000000000 --- a/docs/streams/developer-guide/processor-api.html +++ /dev/null @@ -1,625 +0,0 @@ - - - - - - - - - - - diff --git a/docs/streams/developer-guide/processor-api.md b/docs/streams/developer-guide/processor-api.md new file mode 100644 index 0000000000000..6c0316717ff9e --- /dev/null +++ b/docs/streams/developer-guide/processor-api.md @@ -0,0 +1,352 @@ +--- +title: Processor API +description: +weight: 4 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +The Processor API allows developers to define and connect custom processors and to interact with state stores. With the Processor API, you can define arbitrary stream processors that process one received record at a time, and connect these processors with their associated state stores to compose the processor topology that represents a customized processing logic. + + + + +# Overview + +The Processor API can be used to implement both **stateless** as well as **stateful** operations, where the latter is achieved through the use of state stores. + +**Tip** + +**Combining the DSL and the Processor API:** You can combine the convenience of the DSL with the power and flexibility of the Processor API as described in the section [Applying processors and transformers (Processor API integration)](dsl-api.html#streams-developer-guide-dsl-process). + +For a complete list of available API functionality, see the [Streams](/37/javadoc/org/apache/kafka/streams/package-summary.html) API docs. + +# Defining a Stream Processor + +A [stream processor](../core-concepts.html#streams_processor_node) is a node in the processor topology that represents a single processing step. With the Processor API, you can define arbitrary stream processors that processes one received record at a time, and connect these processors with their associated state stores to compose the processor topology. + +You can define a customized stream processor by implementing the `Processor` interface, which provides the `process()` API method. The `process()` method is called on each of the received records. + +The `Processor` interface also has an `init()` method, which is called by the Kafka Streams library during task construction phase. Processor instances should perform any required initialization in this method. The `init()` method passes in a `ProcessorContext` instance, which provides access to the metadata of the currently processed record, including its source Kafka topic and partition, its corresponding message offset, and further such information. You can also use this context instance to schedule a punctuation function (via `ProcessorContext#schedule()`), to forward a new record as a key-value pair to the downstream processors (via `ProcessorContext#forward()`), and to commit the current processing progress (via `ProcessorContext#commit()`). Any resources you set up in `init()` can be cleaned up in the `close()` method. Note that Kafka Streams may re-use a single `Processor` object by calling `init()` on it again after `close()`. + +The `Processor` interface takes two sets of generic parameters: `KIn, VIn, KOut, VOut`. These define the input and output types that the processor implementation can handle. `KIn` and `VIn` define the key and value types that will be passed to `process()`. Likewise, `KOut` and `VOut` define the forwarded key and value types that `ProcessorContext#forward()` will accept. If your processor does not forward any records at all (or if it only forwards `null` keys or values), a best practice is to set the output generic type argument to `Void`. If it needs to forward multiple types that don't share a common superclass, you will have to set the output generic type argument to `Object`. + +Both the `Processor#process()` and the `ProcessorContext#forward()` methods handle records in the form of the `Record` data class. This class gives you access to the main components of a Kafka record: the key, value, timestamp and headers. When forwarding records, you can use the constructor to create a new `Record` from scratch, or you can use the convenience builder methods to replace one of the `Record`'s properties and copy over the rest. For example, `inputRecord.withValue(newValue)` would copy the key, timestamp, and headers from `inputRecord` while setting the output record's value to `newValue`. Note that this does not mutate `inputRecord`, but instead creates a shallow copy. Beware that this is only a shallow copy, so if you plan to mutate the key, value, or headers elsewhere in the program, you will want to create a deep copy of those fields yourself. + +In addition to handling incoming records via `Processor#process()`, you have the option to schedule periodic invocation (called "punctuation") in your processor's `init()` method by calling `ProcessorContext#schedule()` and passing it a `Punctuator`. The `PunctuationType` determines what notion of time is used for the punctuation scheduling: either [stream-time](../core-concepts.html#streams_time) or wall-clock-time (by default, stream-time is configured to represent event-time via `TimestampExtractor`). When stream-time is used, `punctuate()` is triggered purely by data because stream-time is determined (and advanced forward) by the timestamps derived from the input data. When there is no new input data arriving, stream-time is not advanced and thus `punctuate()` is not called. + +For example, if you schedule a `Punctuator` function every 10 seconds based on `PunctuationType.STREAM_TIME` and if you process a stream of 60 records with consecutive timestamps from 1 (first record) to 60 seconds (last record), then `punctuate()` would be called 6 times. This happens regardless of the time required to actually process those records. `punctuate()` would be called 6 times regardless of whether processing these 60 records takes a second, a minute, or an hour. + +When wall-clock-time (i.e. `PunctuationType.WALL_CLOCK_TIME`) is used, `punctuate()` is triggered purely by the wall-clock time. Reusing the example above, if the `Punctuator` function is scheduled based on `PunctuationType.WALL_CLOCK_TIME`, and if these 60 records were processed within 20 seconds, `punctuate()` is called 2 times (one time every 10 seconds). If these 60 records were processed within 5 seconds, then no `punctuate()` is called at all. Note that you can schedule multiple `Punctuator` callbacks with different `PunctuationType` types within the same processor by calling `ProcessorContext#schedule()` multiple times inside `init()` method. + +**Attention** + +Stream-time is only advanced when Streams processes records. If there are no records to process, or if Streams is waiting for new records due to the [Task Idling](/#streamsconfigs_max.task.idle.ms) configuration, then the stream time will not advance and `punctuate()` will not be triggered if `PunctuationType.STREAM_TIME` was specified. This behavior is independent of the configured timestamp extractor, i.e., using `WallclockTimestampExtractor` does not enable wall-clock triggering of `punctuate()`. + +**Example** + +The following example `Processor` defines a simple word-count algorithm and the following actions are performed: + + * In the `init()` method, schedule the punctuation every 1000 time units (the time unit is normally milliseconds, which in this example would translate to punctuation every 1 second) and retrieve the local state store by its name "Counts". + * In the `process()` method, upon each received record, split the value string into words, and update their counts into the state store (we will talk about this later in this section). + * In the `punctuate()` method, iterate the local state store and send the aggregated counts to the downstream processor (we will talk about downstream processors later in this section), and commit the current stream state. + + + + + public class WordCountProcessor implements Processor { + private KeyValueStore kvStore; + + @Override + public void init(final ProcessorContext context) { + context.schedule(Duration.ofSeconds(1), PunctuationType.STREAM_TIME, timestamp -> { + try (final KeyValueIterator iter = kvStore.all()) { + while (iter.hasNext()) { + final KeyValue entry = iter.next(); + context.forward(new Record<>(entry.key, entry.value.toString(), timestamp)); + } + } + }); + kvStore = context.getStateStore("Counts"); + } + + @Override + public void process(final Record record) { + final String[] words = record.value().toLowerCase(Locale.getDefault()).split("\W+"); + + for (final String word : words) { + final Integer oldValue = kvStore.get(word); + + if (oldValue == null) { + kvStore.put(word, 1); + } else { + kvStore.put(word, oldValue + 1); + } + } + } + + @Override + public void close() { + // close any resources managed by this processor + // Note: Do not close any StateStores as these are managed by the library + } + } + +**Note** + +**Stateful processing with state stores:** The `WordCountProcessor` defined above can access the currently received record in its `process()` method, and it can leverage state stores to maintain processing states to, for example, remember recently arrived records for stateful processing needs like aggregations and joins. For more information, see the state stores documentation. + +# Unit Testing Processors + +Kafka Streams comes with a `test-utils` module to help you write unit tests for your processors [here](testing.html#unit-testing-processors). + +# State Stores + +To implement a **stateful** `Processor` or `Transformer`, you must provide one or more state stores to the processor or transformer (_stateless_ processors or transformers do not need state stores). State stores can be used to remember recently received input records, to track rolling aggregates, to de-duplicate input records, and more. Another feature of state stores is that they can be [interactively queried](interactive-queries.html#streams-developer-guide-interactive-queries) from other applications, such as a NodeJS-based dashboard or a microservice implemented in Scala or Go. + +The available state store types in Kafka Streams have fault tolerance enabled by default. + +## Defining and creating a State Store + +You can either use one of the available store types or implement your own custom store type. It's common practice to leverage an existing store type via the `Stores` factory. + +Note that, when using Kafka Streams, you normally don't create or instantiate state stores directly in your code. Rather, you define state stores indirectly by creating a so-called `StoreBuilder`. This builder is used by Kafka Streams as a factory to instantiate the actual state stores locally in application instances when and where needed. + +The following store types are available out of the box. + +Store Type | Storage Engine | Fault-tolerant? | Description +---|---|---|--- +Persistent `KeyValueStore` | RocksDB | Yes (enabled by default) | + + * **The recommended store type for most use cases.** + * Stores its data on local disk. + * Storage capacity: managed local state can be larger than the memory (heap space) of an application instance, but must fit into the available local disk space. + * RocksDB settings can be fine-tuned, see [RocksDB configuration](config-streams.html#streams-developer-guide-rocksdb-config). + * Available [store variants](/37/javadoc/org/apache/kafka/streams/state/Stores.html#persistentKeyValueStore\(java.lang.String\)): timestamped key-value store, versioned key-value store, time window key-value store, session window key-value store. + * Use [persistentTimestampedKeyValueStore](/37/javadoc/org/apache/kafka/streams/state/Stores.html#persistentTimestampedKeyValueStore\(java.lang.String\)) when you need a persistent key-(value/timestamp) store that supports put/get/delete and range queries. + * Use [persistentVersionedKeyValueStore](/37/javadoc/org/apache/kafka/streams/state/Stores.html#persistentVersionedKeyValueStore\(java.lang.String,java.time.Duration\)) when you need a persistent, versioned key-(value/timestamp) store that supports put/get/delete and timestamped get operations. + * Use [persistentWindowStore](/37/javadoc/org/apache/kafka/streams/state/Stores.html#persistentWindowStore\(java.lang.String,java.time.Duration,java.time.Duration,boolean\)) or [persistentTimestampedWindowStore](/37/javadoc/org/apache/kafka/streams/state/Stores.html#persistentTimestampedWindowStore\(java.lang.String,java.time.Duration,java.time.Duration,boolean\)) when you need a persistent timeWindowedKey-value or timeWindowedKey-(value/timestamp) store, respectively. + * Use [persistentSessionStore](/37/javadoc/org/apache/kafka/streams/state/Stores.html#persistentSessionStore\(java.lang.String,java.time.Duration\)) when you need a persistent sessionWindowedKey-value store. + + + + + // Creating a persistent key-value store: + // here, we create a `KeyValueStore` named "persistent-counts". + import org.apache.kafka.streams.state.StoreBuilder; + import org.apache.kafka.streams.state.Stores; + + // Using a `KeyValueStoreBuilder` to build a `KeyValueStore`. + StoreBuilder> countStoreSupplier = + Stores.keyValueStoreBuilder( + Stores.persistentKeyValueStore("persistent-counts"), + Serdes.String(), + Serdes.Long()); + KeyValueStore countStore = countStoreSupplier.build(); + +In-memory `KeyValueStore` | - | Yes (enabled by default) | + + * Stores its data in memory. + * Storage capacity: managed local state must fit into memory (heap space) of an application instance. + * Useful when application instances run in an environment where local disk space is either not available or local disk space is wiped in-between app instance restarts. + * Available [store variants](/37/javadoc/org/apache/kafka/streams/state/Stores.html#inMemoryKeyValueStore-java.lang.String-): time window key-value store, session window key-value store. + * Use [TimestampedKeyValueStore](/37/javadoc/org/apache/kafka/streams/state/TimestampedKeyValueStore.html) when you need a key-(value/timestamp) store that supports put/get/delete and range queries. + * Use [TimestampedWindowStore](/37/javadoc/org/apache/kafka/streams/state/TimestampedWindowStore.html) when you need to store windowedKey-(value/timestamp) pairs. + * There is no built-in in-memory, versioned key-value store at this time. + + + + + // Creating an in-memory key-value store: + // here, we create a `KeyValueStore` named "inmemory-counts". + import org.apache.kafka.streams.state.StoreBuilder; + import org.apache.kafka.streams.state.Stores; + + // Using a `KeyValueStoreBuilder` to build a `KeyValueStore`. + StoreBuilder> countStoreSupplier = + Stores.keyValueStoreBuilder( + Stores.inMemoryKeyValueStore("inmemory-counts"), + Serdes.String(), + Serdes.Long()); + KeyValueStore countStore = countStoreSupplier.build(); + +## Fault-tolerant State Stores + +To make state stores fault-tolerant and to allow for state store migration without data loss, a state store can be continuously backed up to a Kafka topic behind the scenes. For example, to migrate a stateful stream task from one machine to another when [elastically adding or removing capacity from your application](running-app.html#streams-developer-guide-execution-scaling). This topic is sometimes referred to as the state store's associated _changelog topic_ , or its _changelog_. For example, if you experience machine failure, the state store and the application's state can be fully restored from its changelog. You can enable or disable this backup feature for a state store. + +Fault-tolerant state stores are backed by a [compacted](https://kafka.apache.org/documentation.html#compaction) changelog topic. The purpose of compacting this topic is to prevent the topic from growing indefinitely, to reduce the storage consumed in the associated Kafka cluster, and to minimize recovery time if a state store needs to be restored from its changelog topic. + +Fault-tolerant windowed state stores are backed by a topic that uses both compaction and deletion. Because of the structure of the message keys that are being sent to the changelog topics, this combination of deletion and compaction is required for the changelog topics of window stores. For window stores, the message keys are composite keys that include the "normal" key and window timestamps. For these types of composite keys it would not be sufficient to only enable compaction to prevent a changelog topic from growing out of bounds. With deletion enabled, old windows that have expired will be cleaned up by Kafka's log cleaner as the log segments expire. The default retention setting is `Windows#maintainMs()` \+ 1 day. You can override this setting by specifying `StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG` in the `StreamsConfig`. + +When you open an `Iterator` from a state store you must call `close()` on the iterator when you are done working with it to reclaim resources; or you can use the iterator from within a try-with-resources statement. If you do not close an iterator, you may encounter an OOM error. + +## Enable or Disable Fault Tolerance of State Stores (Store Changelogs) + +You can enable or disable fault tolerance for a state store by enabling or disabling the change logging of the store through `enableLogging()` and `disableLogging()`. You can also fine-tune the associated topic's configuration if needed. + +Example for disabling fault-tolerance: + + + import org.apache.kafka.streams.state.StoreBuilder; + import org.apache.kafka.streams.state.Stores; + + StoreBuilder> countStoreSupplier = Stores.keyValueStoreBuilder( + Stores.persistentKeyValueStore("Counts"), + Serdes.String(), + Serdes.Long()) + .withLoggingDisabled(); // disable backing up the store to a changelog topic + +Attention + +If the changelog is disabled then the attached state store is no longer fault tolerant and it can't have any [standby replicas](config-streams.html#streams-developer-guide-standby-replicas). + +Here is an example for enabling fault tolerance, with additional changelog-topic configuration: You can add any log config from [kafka.log.LogConfig](https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/log/LogConfig.scala). Unrecognized configs will be ignored. + + + import org.apache.kafka.streams.state.StoreBuilder; + import org.apache.kafka.streams.state.Stores; + + Map changelogConfig = new HashMap(); + // override min.insync.replicas + changelogConfig.put(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "1") + + StoreBuilder> countStoreSupplier = Stores.keyValueStoreBuilder( + Stores.persistentKeyValueStore("Counts"), + Serdes.String(), + Serdes.Long()) + .withLoggingEnabled(changelogConfig); // enable changelogging, with custom changelog settings + +## Timestamped State Stores + +KTables always store timestamps by default. A timestamped state store improves stream processing semantics and enables handling out-of-order data in source KTables, detecting out-of-order joins and aggregations, and getting the timestamp of the latest update in an Interactive Query. + +You can query timestamped state stores both with and without a timestamp. + +**Upgrade note:** All users upgrade with a single rolling bounce per instance. + + * For Processor API users, nothing changes in existing applications, and you have the option of using the timestamped stores. + * For DSL operators, store data is upgraded lazily in the background. + * No upgrade happens if you provide a custom XxxBytesStoreSupplier, but you can opt-in by implementing the [TimestampedBytesStore](/37/javadoc/org/apache/kafka/streams/state/TimestampedBytesStore.html) interface. In this case, the old format is retained, and Streams uses a proxy store that removes/adds timestamps on read/write. + + + +## Versioned Key-Value State Stores + +Versioned key-value state stores are available since Kafka Streams 3.5. Rather than storing a single record version (value and timestamp) per key, versioned state stores may store multiple record versions per key. This allows versioned state stores to support timestamped retrieval operations to return the latest record (per key) as of a specified timestamp. + +You can create a persistent, versioned state store by passing a [VersionedBytesStoreSupplier](/37/javadoc/org/apache/kafka/streams/state/Stores.html#persistentVersionedKeyValueStore\(java.lang.String,java.time.Duration\)) to the [versionedKeyValueStoreBuilder](/37/javadoc/org/apache/kafka/streams/state/Stores.html#versionedKeyValueStoreBuilder\(java.lang.String,java.time.Duration\)), or by implementing your own [VersionedKeyValueStore](/37/javadoc/org/apache/kafka/streams/state/VersionedKeyValueStore.html). + +Each versioned store has an associated, fixed-duration _history retention_ parameter which specifies long old record versions should be kept for. In particular, a versioned store guarantees to return accurate results for timestamped retrieval operations where the timestamp being queried is within history retention of the current observed stream time. + +History retention also doubles as its _grace period_ , which determines how far back in time out-of-order writes to the store will be accepted. A versioned store will not accept writes (inserts, updates, or deletions) if the timestamp associated with the write is older than the current observed stream time by more than the grace period. Stream time in this context is tracked per-partition, rather than per-key, which means it's important that grace period (i.e., history retention) be set high enough to accommodate a record with one key arriving out-of-order relative to a record for another key. + +Because the memory footprint of versioned key-value stores is higher than that of non-versioned key-value stores, you may want to adjust your [RocksDB memory settings](memory-mgmt.html#streams-developer-guide-memory-management-rocksdb) accordingly. Benchmarking your application with versioned stores is also advised as performance is expected to be worse than when using non-versioned stores. + +Versioned stores do not support caching or interactive queries at this time. Also, window stores and global tables may not be versioned. + +**Upgrade note:** Versioned state stores are opt-in only; no automatic upgrades from non-versioned to versioned stores will take place. + +Upgrades are supported from persistent, non-versioned key-value stores to persistent, versioned key-value stores as long as the original store has the same changelog topic format as the versioned store being upgraded to. Both persistent [key-value stores](/37/javadoc/org/apache/kafka/streams/state/Stores.html#persistentKeyValueStore\(java.lang.String\)) and [timestamped key-value stores](/37/javadoc/org/apache/kafka/streams/state/Stores.html#persistentTimestampedKeyValueStore\(java.lang.String\)) share the same changelog topic format as [persistent versioned key-value stores](/37/javadoc/org/apache/kafka/streams/state/Stores.html#persistentVersionedKeyValueStore\(java.lang.String,java.time.Duration\)), and therefore both are eligible for upgrades. + +If you wish to upgrade an application using persistent, non-versioned key-value stores to use persistent, versioned key-value stores instead, you can perform the following procedure: + + * Stop all application instances, and [clear any local state directories](app-reset-tool.html#streams-developer-guide-reset-local-environment) for the store(s) being upgraded. + * Update your application code to use versioned stores where desired. + * Update your changelog topic configs, for the relevant state stores, to set the value of `min.compaction.lag.ms` to be at least your desired history retention. History retention plus one day is recommended as buffer for the use of broker wall clock time during compaction. + * Restart your application instances and allow time for the versioned stores to rebuild state from changelog. + + + +## Implementing Custom State Stores + +You can use the built-in state store types or implement your own. The primary interface to implement for the store is `org.apache.kafka.streams.processor.StateStore`. Kafka Streams also has a few extended interfaces such as `KeyValueStore` and `VersionedKeyValueStore`. + +Note that your customized `org.apache.kafka.streams.processor.StateStore` implementation also needs to provide the logic on how to restore the state via the `org.apache.kafka.streams.processor.StateRestoreCallback` or `org.apache.kafka.streams.processor.BatchingStateRestoreCallback` interface. Details on how to instantiate these interfaces can be found in the [javadocs](/37/javadoc/org/apache/kafka/streams/processor/StateStore.html). + +You also need to provide a "builder" for the store by implementing the `org.apache.kafka.streams.state.StoreBuilder` interface, which Kafka Streams uses to create instances of your store. + +# Accessing Processor Context + +As we have mentioned in the Defining a Stream Processor section, a `ProcessorContext` control the processing workflow, such as scheduling a punctuation function, and committing the current processed state. + +This object can also be used to access the metadata related with the application like `applicationId`, `taskId`, and `stateDir`, and also record related metadata as `topic`, `partition`, `offset`, `timestamp` and `headers`. + +Here is an example implementation of how to add a new header to the record: + + + public void process(String key, String value) { + + // add a header to the elements + context().headers().add.("key", "value"); + } + +# Connecting Processors and State Stores + +Now that a processor (WordCountProcessor) and the state stores have been defined, you can construct the processor topology by connecting these processors and state stores together by using the `Topology` instance. In addition, you can add source processors with the specified Kafka topics to generate input data streams into the topology, and sink processors with the specified Kafka topics to generate output data streams out of the topology. + +Here is an example implementation: + + + Topology builder = new Topology(); + // add the source processor node that takes Kafka topic "source-topic" as input + builder.addSource("Source", "source-topic") + // add the WordCountProcessor node which takes the source processor as its upstream processor + .addProcessor("Process", () -> new WordCountProcessor(), "Source") + // add the count store associated with the WordCountProcessor processor + .addStateStore(countStoreBuilder, "Process") + // add the sink processor node that takes Kafka topic "sink-topic" as output + // and the WordCountProcessor node as its upstream processor + .addSink("Sink", "sink-topic", "Process"); + +Here is a quick explanation of this example: + + * A source processor node named `"Source"` is added to the topology using the `addSource` method, with one Kafka topic `"source-topic"` fed to it. + * A processor node named `"Process"` with the pre-defined `WordCountProcessor` logic is then added as the downstream processor of the `"Source"` node using the `addProcessor` method. + * A predefined persistent key-value state store is created and associated with the `"Process"` node, using `countStoreBuilder`. + * A sink processor node is then added to complete the topology using the `addSink` method, taking the `"Process"` node as its upstream processor and writing to a separate `"sink-topic"` Kafka topic (note that users can also use another overloaded variant of `addSink` to dynamically determine the Kafka topic to write to for each received record from the upstream processor). + + + +In some cases, it may be more convenient to add and connect a state store at the same time as you add the processor to the topology. This can be done by implementing `ConnectedStoreProvider#stores()` on the `ProcessorSupplier` instead of calling `Topology#addStateStore()`, like this: + + + Topology builder = new Topology(); + // add the source processor node that takes Kafka "source-topic" as input + builder.addSource("Source", "source-topic") + // add the WordCountProcessor node which takes the source processor as its upstream processor. + // the ProcessorSupplier provides the count store associated with the WordCountProcessor + .addProcessor("Process", new ProcessorSupplier() { + public Processor get() { + return new WordCountProcessor(); + } + + public Set> stores() { + final StoreBuilder> countsStoreBuilder = + Stores + .keyValueStoreBuilder( + Stores.persistentKeyValueStore("Counts"), + Serdes.String(), + Serdes.Long() + ); + return Collections.singleton(countsStoreBuilder); + } + }, "Source") + // add the sink processor node that takes Kafka topic "sink-topic" as output + // and the WordCountProcessor node as its upstream processor + .addSink("Sink", "sink-topic", "Process"); + +This allows for a processor to "own" state stores, effectively encapsulating their usage from the user wiring the topology. Multiple processors that share a state store may provide the same store with this technique, as long as the `StoreBuilder` is the same `instance`. + +In these topologies, the `"Process"` stream processor node is considered a downstream processor of the `"Source"` node, and an upstream processor of the `"Sink"` node. As a result, whenever the `"Source"` node forwards a newly fetched record from Kafka to its downstream `"Process"` node, the `WordCountProcessor#process()` method is triggered to process the record and update the associated state store. Whenever `context#forward()` is called in the `WordCountProcessor#punctuate()` method, the aggregate key-value pair will be sent via the `"Sink"` processor node to the Kafka topic `"sink-topic"`. Note that in the `WordCountProcessor` implementation, you must refer to the same store name `"Counts"` when accessing the key-value store, otherwise an exception will be thrown at runtime, indicating that the state store cannot be found. If the state store is not associated with the processor in the `Topology` code, accessing it in the processor's `init()` method will also throw an exception at runtime, indicating the state store is not accessible from this processor. + +Note that the `Topology#addProcessor` function takes a `ProcessorSupplier` as argument, and that the supplier pattern requires that a new `Processor` instance is returned each time `ProcessorSupplier#get()` is 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. So remember not to provide a singleton `Processor` instance to `Topology`. The `ProcessorSupplier` should always generate a new instance each time `ProcessorSupplier#get()` gets called. + +Now that you have fully defined your processor topology in your application, you can proceed to [running the Kafka Streams application](running-app.html#streams-developer-guide-execution). + + * [Documentation](/documentation) + * [Kafka Streams](/streams) + * [Developer Guide](/streams/developer-guide/) + + diff --git a/docs/streams/developer-guide/running-app.html b/docs/streams/developer-guide/running-app.html deleted file mode 100644 index ff3ed75d29010..0000000000000 --- a/docs/streams/developer-guide/running-app.html +++ /dev/null @@ -1,188 +0,0 @@ - - - - - - - - - - - \ No newline at end of file diff --git a/docs/streams/developer-guide/running-app.md b/docs/streams/developer-guide/running-app.md new file mode 100644 index 0000000000000..35a70f88cb2ac --- /dev/null +++ b/docs/streams/developer-guide/running-app.md @@ -0,0 +1,87 @@ +--- +title: Running Streams Applications +description: +weight: 10 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +You can run Java applications that use the Kafka Streams library without any additional configuration or requirements. Kafka Streams also provides the ability to receive notification of the various states of the application. The ability to monitor the runtime status is discussed in [the monitoring guide](/#kafka_streams_monitoring). + + + + +# Starting a Kafka Streams application + +You can package your Java application as a fat JAR file and then start the application like this: + + + # Start the application in class `com.example.MyStreamsApp` + # from the fat JAR named `path-to-app-fatjar.jar`. + $ java -cp path-to-app-fatjar.jar com.example.MyStreamsApp + +When you start your application you are launching a Kafka Streams instance of your application. You can run multiple instances of your application. A common scenario is that there are multiple instances of your application running in parallel. For more information, see [Parallelism Model](../architecture.html#streams_architecture_tasks). + +When the application instance starts running, the defined processor topology will be initialized as one or more stream tasks. If the processor topology defines any state stores, these are also constructed during the initialization period. For more information, see the State restoration during workload rebalance section). + +# Elastic scaling of your application + +Kafka Streams makes your stream processing applications elastic and scalable. You can add and remove processing capacity dynamically during application runtime without any downtime or data loss. This makes your applications resilient in the face of failures and for allows you to perform maintenance as needed (e.g. rolling upgrades). + +For more information about this elasticity, see the [Parallelism Model](../architecture.html#streams_architecture_tasks) section. Kafka Streams leverages the Kafka group management functionality, which is built right into the [Kafka wire protocol](https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol). It is the foundation that enables the elasticity of Kafka Streams applications: members of a group coordinate and collaborate jointly on the consumption and processing of data in Kafka. Additionally, Kafka Streams provides stateful processing and allows for fault-tolerant state in environments where application instances may come and go at any time. + +## Adding capacity to your application + +If you need more processing capacity for your stream processing application, you can simply start another instance of your stream processing application, e.g. on another machine, in order to scale out. The instances of your application will become aware of each other and automatically begin to share the processing work. More specifically, what will be handed over from the existing instances to the new instances is (some of) the stream tasks that have been run by the existing instances. Moving stream tasks from one instance to another results in moving the processing work plus any internal state of these stream tasks (the state of a stream task will be re-created in the target instance by restoring the state from its corresponding changelog topic). + +The various instances of your application each run in their own JVM process, which means that each instance can leverage all the processing capacity that is available to their respective JVM process (minus the capacity that any non-Kafka-Streams part of your application may be using). This explains why running additional instances will grant your application additional processing capacity. The exact capacity you will be adding by running a new instance depends of course on the environment in which the new instance runs: available CPU cores, available main memory and Java heap space, local storage, network bandwidth, and so on. Similarly, if you stop any of the running instances of your application, then you are removing and freeing up the respective processing capacity. + +![](/37/images/streams-elastic-scaling-1.png) + +Before adding capacity: only a single instance of your Kafka Streams application is running. At this point the corresponding Kafka consumer group of your application contains only a single member (this instance). All data is being read and processed by this single instance. + +![](/37/images/streams-elastic-scaling-2.png) + +After adding capacity: now two additional instances of your Kafka Streams application are running, and they have automatically joined the application's Kafka consumer group for a total of three current members. These three instances are automatically splitting the processing work between each other. The splitting is based on the Kafka topic partitions from which data is being read. + +## Removing capacity from your application + +To remove processing capacity, you can stop running stream processing application instances (e.g., shut down two of the four instances), it will automatically leave the application’s consumer group, and the remaining instances of your application will automatically take over the processing work. The remaining instances take over the stream tasks that were run by the stopped instances. Moving stream tasks from one instance to another results in moving the processing work plus any internal state of these stream tasks. The state of a stream task is recreated in the target instance from its changelog topic. + +![](/37/images/streams-elastic-scaling-3.png) + +## State restoration during workload rebalance + +When a task is migrated, the task processing state is fully restored before the application instance resumes processing. This guarantees the correct processing results. In Kafka Streams, state restoration is usually done by replaying the corresponding changelog topic to reconstruct the state store. To minimize changelog-based restoration latency by using replicated local state stores, you can specify `num.standby.replicas`. When a stream task is initialized or re-initialized on the application instance, its state store is restored like this: + + * If no local state store exists, the changelog is replayed from the earliest to the current offset. This reconstructs the local state store to the most recent snapshot. + * If a local state store exists, the changelog is replayed from the previously checkpointed offset. The changes are applied and the state is restored to the most recent snapshot. This method takes less time because it is applying a smaller portion of the changelog. + + + +For more information, see [Standby Replicas](config-streams.html#num-standby-replicas). + +As of version 2.6, Streams will now do most of a task's restoration in the background through warmup replicas. These will be assigned to instances that need to restore a lot of state for a task. A stateful active task will only be assigned to an instance once its state is within the configured [`acceptable.recovery.lag`](config-streams.html#acceptable-recovery-lag), if one exists. This means that most of the time, a task migration will **not** result in downtime for that task. It will remain active on the instance that's already caught up, while the instance that it's being migrated to works on restoring the state. Streams will [regularly probe](config-streams.html#probing-rebalance-interval-ms) for warmup tasks that have finished restoring and transition them to active tasks when ready. + +Note, the one exception to this task availability is if none of the instances have a caught up version of that task. In that case, we have no choice but to assign the active task to an instance that is not caught up and will have to block further processing on restoration of the task's state from the changelog. If high availability is important for your application, you are highly recommended to enable standbys. + +## Determining how many application instances to run + +The parallelism of a Kafka Streams application is primarily determined by how many partitions the input topics have. For example, if your application reads from a single topic that has ten partitions, then you can run up to ten instances of your applications. You can run further instances, but these will be idle. + +The number of topic partitions is the upper limit for the parallelism of your Kafka Streams application and for the number of running instances of your application. + +To achieve balanced workload processing across application instances and to prevent processing hotpots, you should distribute data and processing workloads: + + * Data should be equally distributed across topic partitions. For example, if two topic partitions each have 1 million messages, this is better than a single partition with 2 million messages and none in the other. + * Processing workload should be equally distributed across topic partitions. For example, if the time to process messages varies widely, then it is better to spread the processing-intensive messages across partitions rather than storing these messages within the same partition. + + + + * [Documentation](/documentation) + * [Kafka Streams](/streams) + * [Developer Guide](/streams/developer-guide/) + + diff --git a/docs/streams/developer-guide/security.html b/docs/streams/developer-guide/security.html deleted file mode 100644 index 74319f9e33c59..0000000000000 --- a/docs/streams/developer-guide/security.html +++ /dev/null @@ -1,190 +0,0 @@ - - - - - - - - - - - diff --git a/docs/streams/developer-guide/security.md b/docs/streams/developer-guide/security.md new file mode 100644 index 0000000000000..21b23136fea58 --- /dev/null +++ b/docs/streams/developer-guide/security.md @@ -0,0 +1,95 @@ +--- +title: Streams Security +description: +weight: 12 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + + + + +Kafka Streams natively integrates with the [Kafka's security features](../../../documentation.html#security) and supports all of the client-side security features in Kafka. Streams leverages the [Java Producer and Consumer API](../../../documentation.html#api). + +To secure your Stream processing applications, configure the security settings in the corresponding Kafka producer and consumer clients, and then specify the corresponding configuration settings in your Kafka Streams application. + +Kafka supports cluster encryption and authentication, including a mix of authenticated and unauthenticated, and encrypted and non-encrypted clients. Using security is optional. + +Here a few relevant client-side security features: + +Encrypt data-in-transit between your applications and Kafka brokers + You can enable the encryption of the client-server communication between your applications and the Kafka brokers. For example, you can configure your applications to always use encryption when reading and writing data to and from Kafka. This is critical when reading and writing data across security domains such as internal network, public internet, and partner networks. +Client authentication + You can enable client authentication for connections from your application to Kafka brokers. For example, you can define that only specific applications are allowed to connect to your Kafka cluster. +Client authorization + You can enable client authorization of read and write operations by your applications. For example, you can define that only specific applications are allowed to read from a Kafka topic. You can also restrict write access to Kafka topics to prevent data pollution or fraudulent activities. + +For more information about the security features in Apache Kafka, see [Kafka Security](../../../documentation.html#security). + +# Required ACL setting for secure Kafka clusters + +Kafka clusters can use ACLs to control access to resources (like the ability to create topics), and for such clusters each client, including Kafka Streams, is required to authenticate as a particular user in order to be authorized with appropriate access. In particular, when Streams applications are run against a secured Kafka cluster, the principal running the application must have the ACL set so that the application has the permissions to create, read and write [internal topics](manage-topics.html#streams-developer-guide-topics-internal). + +To avoid providing this permission to your application, you can create the required internal topics manually. If the internal topics exist, Kafka Streams will not try to recreate them. Note, that the internal repartition and changelog topics must be created with the correct number of partitions--otherwise, Kafka Streams will fail on startup. The topics must be created with the same number of partitions as your input topic, or if there are multiple topics, the maximum number of partitions across all input topics. Additionally, changelog topics must be created with log compaction enabled--otherwise, your application might lose data. For changelog topics for windowed KTables, apply "delete,compact" and set the retention time based on the corresponding store retention time. To avoid premature deletion, add a delta to the store retention time. By default, Kafka Streams adds 24 hours to the store retention time. You can find out more about the names of the required internal topics via `Topology#describe()`. All internal topics follow the naming pattern `--` where the `suffix` is either `repartition` or `changelog`. Note, that there is no guarantee about this naming pattern in future releases--it's not part of the public API. + +Since all internal topics as well as the embedded consumer group name are prefixed with the [application id](/37/streams/developer-guide/config-streams.html#required-configuration-parameters), it is recommended to use ACLs on prefixed resource pattern to configure control lists to allow client to manage all topics and consumer groups started with this prefix as `--resource-pattern-type prefixed --topic your.application.id --operation All ` (see [KIP-277](https://cwiki.apache.org/confluence/display/KAFKA/KIP-277+-+Fine+Grained+ACL+for+CreateTopics+API) and [KIP-290](https://cwiki.apache.org/confluence/display/KAFKA/KIP-290%3A+Support+for+Prefixed+ACLs) for details). + +# Security example + +The purpose is to configure a Kafka Streams application to enable client authentication and encrypt data-in-transit when communicating with its Kafka cluster. + +This example assumes that the Kafka brokers in the cluster already have their security setup and that the necessary SSL certificates are available to the application in the local filesystem locations. For example, if you are using Docker then you must also include these SSL certificates in the correct locations within the Docker image. + +The snippet below shows the settings to enable client authentication and SSL encryption for data-in-transit between your Kafka Streams application and the Kafka cluster it is reading and writing from: + + + # Essential security settings to enable client authentication and SSL encryption + bootstrap.servers=kafka.example.com:9093 + security.protocol=SSL + ssl.truststore.location=/etc/security/tls/kafka.client.truststore.jks + ssl.truststore.password=test1234 + ssl.keystore.location=/etc/security/tls/kafka.client.keystore.jks + ssl.keystore.password=test1234 + ssl.key.password=test1234 + +Configure these settings in the application for your `Properties` instance. These settings will encrypt any data-in-transit that is being read from or written to Kafka, and your application will authenticate itself against the Kafka brokers that it is communicating with. Note that this example does not cover client authorization. + + + // Code of your Java application that uses the Kafka Streams library + Properties settings = new Properties(); + settings.put(StreamsConfig.APPLICATION_ID_CONFIG, "secure-kafka-streams-app"); + // Where to find secure Kafka brokers. Here, it's on port 9093. + settings.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka.example.com:9093"); + // + // ...further non-security related settings may follow here... + // + // Security settings. + // 1. These settings must match the security settings of the secure Kafka cluster. + // 2. The SSL trust store and key store files must be locally accessible to the application. + settings.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SSL"); + settings.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, "/etc/security/tls/kafka.client.truststore.jks"); + settings.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, "test1234"); + settings.put(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, "/etc/security/tls/kafka.client.keystore.jks"); + settings.put(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, "test1234"); + settings.put(SslConfigs.SSL_KEY_PASSWORD_CONFIG, "test1234"); + +If you incorrectly configure a security setting in your application, it will fail at runtime, typically right after you start it. For example, if you enter an incorrect password for the `ssl.keystore.password` setting, an error message similar to this would be logged and then the application would terminate: + + + # Misconfigured ssl.keystore.password + Exception in thread "main" org.apache.kafka.common.KafkaException: Failed to construct kafka producer + [...snip...] + Caused by: org.apache.kafka.common.KafkaException: org.apache.kafka.common.KafkaException: + java.io.IOException: Keystore was tampered with, or password was incorrect + [...snip...] + Caused by: java.security.UnrecoverableKeyException: Password verification failed + +Monitor your Kafka Streams application log files for such error messages to spot any misconfigured applications quickly. + + * [Documentation](/documentation) + * [Kafka Streams](/streams) + * [Developer Guide](/streams/developer-guide/) + + diff --git a/docs/streams/developer-guide/testing.html b/docs/streams/developer-guide/testing.html deleted file mode 100644 index b5fadb12b3d3e..0000000000000 --- a/docs/streams/developer-guide/testing.html +++ /dev/null @@ -1,400 +0,0 @@ - - - - - - - - - - - diff --git a/docs/streams/developer-guide/testing.md b/docs/streams/developer-guide/testing.md new file mode 100644 index 0000000000000..e472f0abab05a --- /dev/null +++ b/docs/streams/developer-guide/testing.md @@ -0,0 +1,315 @@ +--- +title: Testing a Streams Application +description: +weight: 7 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +# Testing Kafka Streams + + + + +# Importing the test utilities + +To test a Kafka Streams application, Kafka provides a test-utils artifact that can be added as regular dependency to your test code base. Example `pom.xml` snippet when using Maven: + + + + org.apache.kafka + kafka-streams-test-utils + 3.7.2 + test + + +# Testing a Streams application + +The test-utils package provides a `TopologyTestDriver` that can be used pipe data through a `Topology` that is either assembled manually using Processor API or via the DSL using `StreamsBuilder`. The test driver simulates the library runtime that continuously fetches records from input topics and processes them by traversing the topology. You can use the test driver to verify that your specified processor topology computes the correct result with the manually piped in data records. The test driver captures the results records and allows to query its embedded state stores. + + + // Processor API + Topology topology = new Topology(); + topology.addSource("sourceProcessor", "input-topic"); + topology.addProcessor("processor", ..., "sourceProcessor"); + topology.addSink("sinkProcessor", "output-topic", "processor"); + // or + // using DSL + StreamsBuilder builder = new StreamsBuilder(); + builder.stream("input-topic").filter(...).to("output-topic"); + Topology topology = builder.build(); + + // create test driver + TopologyTestDriver testDriver = new TopologyTestDriver(topology); + +With the test driver you can create `TestInputTopic` giving topic name and the corresponding serializers. `TestInputTopic` provides various methods to pipe new message values, keys and values, or list of KeyValue objects. + + + TestInputTopic inputTopic = testDriver.createInputTopic("input-topic", stringSerde.serializer(), longSerde.serializer()); + inputTopic.pipeInput("key", 42L); + +To verify the output, you can use `TestOutputTopic` where you configure the topic and the corresponding deserializers during initialization. It offers helper methods to read only certain parts of the result records or the collection of records. For example, you can validate returned `KeyValue` with standard assertions if you only care about the key and value, but not the timestamp of the result record. + + + TestOutputTopic outputTopic = testDriver.createOutputTopic("output-topic", stringSerde.deserializer(), longSerde.deserializer()); + assertThat(outputTopic.readKeyValue(), equalTo(new KeyValue<>("key", 42L))); + +`TopologyTestDriver` supports punctuations, too. Event-time punctuations are triggered automatically based on the processed records' timestamps. Wall-clock-time punctuations can also be triggered by advancing the test driver's wall-clock-time (the driver mocks wall-clock-time internally to give users control over it). + + + testDriver.advanceWallClockTime(Duration.ofSeconds(20)); + +Additionally, you can access state stores via the test driver before or after a test. Accessing stores before a test is useful to pre-populate a store with some initial values. After data was processed, expected updates to the store can be verified. + + + KeyValueStore store = testDriver.getKeyValueStore("store-name"); + +Note, that you should always close the test driver at the end to make sure all resources are release properly. + + + testDriver.close(); + +## Example + +The following example demonstrates how to use the test driver and helper classes. The example creates a topology that computes the maximum value per key using a key-value-store. While processing, no output is generated, but only the store is updated. Output is only sent downstream based on event-time and wall-clock punctuations. + + + private TopologyTestDriver testDriver; + private TestInputTopic inputTopic; + private TestOutputTopic outputTopic; + private KeyValueStore store; + + private Serde stringSerde = new Serdes.StringSerde(); + private Serde longSerde = new Serdes.LongSerde(); + + @Before + public void setup() { + Topology topology = new Topology(); + topology.addSource("sourceProcessor", "input-topic"); + topology.addProcessor("aggregator", new CustomMaxAggregatorSupplier(), "sourceProcessor"); + topology.addStateStore( + Stores.keyValueStoreBuilder( + Stores.inMemoryKeyValueStore("aggStore"), + Serdes.String(), + Serdes.Long()).withLoggingDisabled(), // need to disable logging to allow store pre-populating + "aggregator"); + topology.addSink("sinkProcessor", "result-topic", "aggregator"); + + // setup test driver + Properties props = new Properties(); + props.setProperty(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); + props.setProperty(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Long().getClass().getName()); + testDriver = new TopologyTestDriver(topology, props); + + // setup test topics + inputTopic = testDriver.createInputTopic("input-topic", stringSerde.serializer(), longSerde.serializer()); + outputTopic = testDriver.createOutputTopic("result-topic", stringSerde.deserializer(), longSerde.deserializer()); + + // pre-populate store + store = testDriver.getKeyValueStore("aggStore"); + store.put("a", 21L); + } + + @After + public void tearDown() { + testDriver.close(); + } + + @Test + public void shouldFlushStoreForFirstInput() { + inputTopic.pipeInput("a", 1L); + assertThat(outputTopic.readKeyValue(), equalTo(new KeyValue<>("a", 21L))); + assertThat(outputTopic.isEmpty(), is(true)); + } + + @Test + public void shouldNotUpdateStoreForSmallerValue() { + inputTopic.pipeInput("a", 1L); + assertThat(store.get("a"), equalTo(21L)); + assertThat(outputTopic.readKeyValue(), equalTo(new KeyValue<>("a", 21L))); + assertThat(outputTopic.isEmpty(), is(true)); + } + + @Test + public void shouldNotUpdateStoreForLargerValue() { + inputTopic.pipeInput("a", 42L); + assertThat(store.get("a"), equalTo(42L)); + assertThat(outputTopic.readKeyValue(), equalTo(new KeyValue<>("a", 42L))); + assertThat(outputTopic.isEmpty(), is(true)); + } + + @Test + public void shouldUpdateStoreForNewKey() { + inputTopic.pipeInput("b", 21L); + assertThat(store.get("b"), equalTo(21L)); + assertThat(outputTopic.readKeyValue(), equalTo(new KeyValue<>("a", 21L))); + assertThat(outputTopic.readKeyValue(), equalTo(new KeyValue<>("b", 21L))); + assertThat(outputTopic.isEmpty(), is(true)); + } + + @Test + public void shouldPunctuateIfEvenTimeAdvances() { + final Instant recordTime = Instant.now(); + inputTopic.pipeInput("a", 1L, recordTime); + assertThat(outputTopic.readKeyValue(), equalTo(new KeyValue<>("a", 21L))); + + inputTopic.pipeInput("a", 1L, recordTime); + assertThat(outputTopic.isEmpty(), is(true)); + + inputTopic.pipeInput("a", 1L, recordTime.plusSeconds(10L)); + assertThat(outputTopic.readKeyValue(), equalTo(new KeyValue<>("a", 21L))); + assertThat(outputTopic.isEmpty(), is(true)); + } + + @Test + public void shouldPunctuateIfWallClockTimeAdvances() { + testDriver.advanceWallClockTime(Duration.ofSeconds(60)); + assertThat(outputTopic.readKeyValue(), equalTo(new KeyValue<>("a", 21L))); + assertThat(outputTopic.isEmpty(), is(true)); + } + + public class CustomMaxAggregatorSupplier implements ProcessorSupplier { + @Override + public Processor get() { + return new CustomMaxAggregator(); + } + } + + public class CustomMaxAggregator implements Processor { + ProcessorContext context; + private KeyValueStore store; + + @SuppressWarnings("unchecked") + @Override + public void init(ProcessorContext context) { + this.context = context; + context.schedule(Duration.ofSeconds(60), PunctuationType.WALL_CLOCK_TIME, time -> flushStore()); + context.schedule(Duration.ofSeconds(10), PunctuationType.STREAM_TIME, time -> flushStore()); + store = (KeyValueStore) context.getStateStore("aggStore"); + } + + @Override + public void process(String key, Long value) { + Long oldValue = store.get(key); + if (oldValue == null || value > oldValue) { + store.put(key, value); + } + } + + private void flushStore() { + KeyValueIterator it = store.all(); + while (it.hasNext()) { + KeyValue next = it.next(); + context.forward(next.key, next.value); + } + } + + @Override + public void close() {} + } + +# Unit Testing Processors + +If you [write a Processor](processor-api.html), you will want to test it. + +Because the `Processor` forwards its results to the context rather than returning them, Unit testing requires a mocked context capable of capturing forwarded data for inspection. For this reason, we provide a `MockProcessorContext` in `test-utils`. + +**Construction** + +To begin with, instantiate your processor and initialize it with the mock context: + + + final Processor processorUnderTest = ...; + final MockProcessorContext context = new MockProcessorContext(); + processorUnderTest.init(context); + +If you need to pass configuration to your processor or set the default serdes, you can create the mock with config: + + + final Properties props = new Properties(); + props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass()); + props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Long().getClass()); + props.put("some.other.config", "some config value"); + final MockProcessorContext context = new MockProcessorContext(props); + +**Captured data** + +The mock will capture any values that your processor forwards. You can make assertions on them: + + + processorUnderTest.process("key", "value"); + + final Iterator forwarded = context.forwarded().iterator(); + assertEquals(forwarded.next().keyValue(), new KeyValue<>(..., ...)); + assertFalse(forwarded.hasNext()); + + // you can reset forwards to clear the captured data. This may be helpful in constructing longer scenarios. + context.resetForwards(); + + assertEquals(context.forwarded().size(), 0); + +If your processor forwards to specific child processors, you can query the context for captured data by child name: + + + final List captures = context.forwarded("childProcessorName"); + +The mock also captures whether your processor has called `commit()` on the context: + + + assertTrue(context.committed()); + + // commit captures can also be reset. + context.resetCommit(); + + assertFalse(context.committed()); + +**Setting record metadata** + +In case your processor logic depends on the record metadata (topic, partition, offset, or timestamp), you can set them on the context, either all together or individually: + + + context.setRecordMetadata("topicName", /*partition*/ 0, /*offset*/ 0L, /*timestamp*/ 0L); + context.setTopic("topicName"); + context.setPartition(0); + context.setOffset(0L); + context.setTimestamp(0L); + +Once these are set, the context will continue returning the same values, until you set new ones. + +**State stores** + +In case your punctuator is stateful, the mock context allows you to register state stores. You're encouraged to use a simple in-memory store of the appropriate type (KeyValue, Windowed, or Session), since the mock context does _not_ manage changelogs, state directories, etc. + + + final KeyValueStore store = + Stores.keyValueStoreBuilder( + Stores.inMemoryKeyValueStore("myStore"), + Serdes.String(), + Serdes.Integer() + ) + .withLoggingDisabled() // Changelog is not supported by MockProcessorContext. + .build(); + store.init(context, store); + context.register(store, /*deprecated parameter*/ false, /*parameter unused in mock*/ null); + +**Verifying punctuators** + +Processors can schedule punctuators to handle periodic tasks. The mock context does _not_ automatically execute punctuators, but it does capture them to allow you to unit test them as well: + + + final MockProcessorContext.CapturedPunctuator capturedPunctuator = context.scheduledPunctuators().get(0); + final long interval = capturedPunctuator.getIntervalMs(); + final PunctuationType type = capturedPunctuator.getType(); + final boolean cancelled = capturedPunctuator.cancelled(); + final Punctuator punctuator = capturedPunctuator.getPunctuator(); + punctuator.punctuate(/*timestamp*/ 0L); + +If you need to write tests involving automatic firing of scheduled punctuators, we recommend creating a simple topology with your processor and using the [`TopologyTestDriver`](testing.html#testing-topologytestdriver). + + * [Documentation](/documentation) + * [Kafka Streams](/streams) + * [Developer Guide](/streams/developer-guide/) + + diff --git a/docs/streams/developer-guide/write-streams-app.md b/docs/streams/developer-guide/write-streams-app.md new file mode 100644 index 0000000000000..57b4e6c5c4cf5 --- /dev/null +++ b/docs/streams/developer-guide/write-streams-app.md @@ -0,0 +1,157 @@ +--- +title: Writing a Streams Application +description: +weight: 1 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + + + + +Any Java or Scala application that makes use of the Kafka Streams library is considered a Kafka Streams application. The computational logic of a Kafka Streams application is defined as a [processor topology](../core-concepts#streams_topology), which is a graph of stream processors (nodes) and streams (edges). + +You can define the processor topology with the Kafka Streams APIs: + +[Kafka Streams DSL](dsl-api.html#streams-developer-guide-dsl) + A high-level API that provides the most common data transformation operations such as `map`, `filter`, `join`, and `aggregations` out of the box. The DSL is the recommended starting point for developers new to Kafka Streams, and should cover many use cases and stream processing needs. If you're writing a Scala application then you can use the [Kafka Streams DSL for Scala](dsl-api.html#scala-dsl) library which removes much of the Java/Scala interoperability boilerplate as opposed to working directly with the Java DSL. +[Processor API](processor-api.html#streams-developer-guide-processor-api) + A low-level API that lets you add and connect processors as well as interact directly with state stores. The Processor API provides you with even more flexibility than the DSL but at the expense of requiring more manual work on the side of the application developer (e.g., more lines of code). + +# Libraries and Maven artifacts + +This section lists the Kafka Streams related libraries that are available for writing your Kafka Streams applications. + +You can define dependencies on the following libraries for your Kafka Streams applications. + +Group ID | Artifact ID | Version | Description +---|---|---|--- +`org.apache.kafka` | `kafka-streams` | `3.7.2` | (Required) Base library for Kafka Streams. +`org.apache.kafka` | `kafka-clients` | `3.7.2` | (Required) Kafka client library. Contains built-in serializers/deserializers. +`org.apache.kafka` | `kafka-streams-scala` | `3.7.2` | (Optional) Kafka Streams DSL for Scala library to write Scala Kafka Streams applications. When not using SBT you will need to suffix the artifact ID with the correct version of Scala your application is using (`_2.12`, `_2.13`) + +**Tip** + +See the section [Data Types and Serialization](datatypes.html#streams-developer-guide-serdes) for more information about Serializers/Deserializers. + +Example `pom.xml` snippet when using Maven: + + + + org.apache.kafka + kafka-streams + 3.7.2 + + + org.apache.kafka + kafka-clients + 3.7.2 + + + org.apache.kafka + kafka-streams-scala_2.13 + 3.7.2 + + +# Using Kafka Streams within your application code + +You can call Kafka Streams from anywhere in your application code, but usually these calls are made within the `main()` method of your application, or some variant thereof. The basic elements of defining a processing topology within your application are described below. + +First, you must create an instance of `KafkaStreams`. + + * The first argument of the `KafkaStreams` constructor takes a topology (either `StreamsBuilder#build()` for the [DSL](dsl-api.html#streams-developer-guide-dsl) or `Topology` for the [Processor API](processor-api.html#streams-developer-guide-processor-api)) that is used to define a topology. + * The second argument is an instance of `java.util.Properties`, which defines the configuration for this specific topology. + + + +Code example: + + + import org.apache.kafka.streams.KafkaStreams; + import org.apache.kafka.streams.kstream.StreamsBuilder; + import org.apache.kafka.streams.processor.Topology; + + // Use the builders to define the actual processing topology, e.g. to specify + // from which input topics to read, which stream operations (filter, map, etc.) + // should be called, and so on. We will cover this in detail in the subsequent + // sections of this Developer Guide. + + StreamsBuilder builder = ...; // when using the DSL + Topology topology = builder.build(); + // + // OR + // + Topology topology = ...; // when using the Processor API + + // Use the configuration to tell your application where the Kafka cluster is, + // which Serializers/Deserializers to use by default, to specify security settings, + // and so on. + Properties props = ...; + + KafkaStreams streams = new KafkaStreams(topology, props); + +At this point, internal structures are initialized, but the processing is not started yet. You have to explicitly start the Kafka Streams thread by calling the `KafkaStreams#start()` method: + + + // Start the Kafka Streams threads + streams.start(); + +If there are other instances of this stream processing application running elsewhere (e.g., on another machine), Kafka Streams transparently re-assigns tasks from the existing instances to the new instance that you just started. For more information, see [Stream Partitions and Tasks](../architecture.html#streams_architecture_tasks) and [Threading Model](../architecture.html#streams_architecture_threads). + +To catch any unexpected exceptions, you can set an `java.lang.Thread.UncaughtExceptionHandler` before you start the application. This handler is called whenever a stream thread is terminated by an unexpected exception: + + + // Java 8+, using lambda expressions + streams.setUncaughtExceptionHandler((Thread thread, Throwable throwable) -> { + // here you should examine the throwable/exception and perform an appropriate action! + }); + + + // Java 7 + streams.setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() { + public void uncaughtException(Thread thread, Throwable throwable) { + // here you should examine the throwable/exception and perform an appropriate action! + } + }); + +To stop the application instance, call the `KafkaStreams#close()` method: + + + // Stop the Kafka Streams threads + streams.close(); + +To allow your application to gracefully shutdown in response to SIGTERM, it is recommended that you add a shutdown hook and call `KafkaStreams#close`. + + * Here is a shutdown hook example in Java 8+: + + // Add shutdown hook to stop the Kafka Streams threads. + // You can optionally provide a timeout to `close`. + Runtime.getRuntime().addShutdownHook(new Thread(streams::close)); + + * Here is a shutdown hook example in Java 7: + + // Add shutdown hook to stop the Kafka Streams threads. + // You can optionally provide a timeout to `close`. + Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() { + @Override + public void run() { + streams.close(); + } + })); + + + + +After an application is stopped, Kafka Streams will migrate any tasks that had been running in this instance to available remaining instances. + +# Testing a Streams application + +Kafka Streams comes with a `test-utils` module to help you test your application [here](testing.html). + + * [Documentation](/documentation) + * [Kafka Streams](/streams) + * [Developer Guide](/streams/developer-guide/) + + diff --git a/docs/streams/developer-guide/write-streams.html b/docs/streams/developer-guide/write-streams.html deleted file mode 100644 index 03bd16328efbe..0000000000000 --- a/docs/streams/developer-guide/write-streams.html +++ /dev/null @@ -1,245 +0,0 @@ - - - - - - - - - - - diff --git a/docs/streams/index.html b/docs/streams/index.html deleted file mode 100644 index 7fd2d7e6ade0e..0000000000000 --- a/docs/streams/index.html +++ /dev/null @@ -1,408 +0,0 @@ - - - - - - - -
- -
- - -
-
-
- - diff --git a/docs/streams/introduction.md b/docs/streams/introduction.md new file mode 100644 index 0000000000000..fe25e39538353 --- /dev/null +++ b/docs/streams/introduction.md @@ -0,0 +1,206 @@ +--- +title: Introduction +description: +weight: 1 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +# Kafka Streams + +## The easiest way to write mission-critical real-time applications and microservices + +Kafka Streams is a client library for building applications and microservices, where the input and output data are stored in Kafka clusters. It combines the simplicity of writing and deploying standard Java and Scala applications on the client side with the benefits of Kafka's server-side cluster technology. + + + + + + + + + + +## Tour of the Streams API + +{{< carousel >}} +{{< carousel-item title="1. Intro to Streams" active="true" >}} +{{% youtube "ni3XPsYC5cQ" %}} +{{< /carousel-item >}} +{{< carousel-item title="2. Creating a Streams Application" >}} +{{% youtube "9ZhsnXM2OVM" %}} +{{< /carousel-item >}} +{{< carousel-item title="3. Transforming Data Pt. 1" >}} +{{% youtube "SYmqwvE8umM" %}} +{{< /carousel-item >}} +{{< carousel-item title="4. Transforming Data Pt. 2" >}} +{{% youtube "Vk55Kl9x_Fw" %}} +{{< /carousel-item >}} +{{< /carousel >}} + + + +* * * + +## Why you'll love using Kafka Streams! + + * Elastic, highly scalable, fault-tolerant + * Deploy to containers, VMs, bare metal, cloud + * Equally viable for small, medium, & large use cases + * Fully integrated with Kafka security + * Write standard Java and Scala applications + * Exactly-once processing semantics + * No separate processing cluster required + * Develop on Mac, Linux, Windows + + + +[Write your first app](/37/streams/tutorial) + +* * * + +## Kafka Streams use cases + + +{{< about/kstreams-users >}} + + +## Hello Kafka Streams + +The code example below implements a WordCount application that is elastic, highly scalable, fault-tolerant, stateful, and ready to run in production at large scale + +{{< tabpane >}} +{{% tab header="Java 8+ Java 7" %}} +```java +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KTable; +import org.apache.kafka.streams.kstream.Materialized; +import org.apache.kafka.streams.kstream.Produced; +import org.apache.kafka.streams.state.KeyValueStore; + +import java.util.Arrays; +import java.util.Properties; + +public class WordCountApplication { + + public static void main(final String[] args) throws Exception { + Properties props = new Properties(); + props.put(StreamsConfig.APPLICATION_ID_CONFIG, "wordcount-application"); + props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka-broker1:9092"); + props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass()); + props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); + + StreamsBuilder builder = new StreamsBuilder(); + KStream textLines = builder.stream("TextLinesTopic"); + KTable wordCounts = textLines + .flatMapValues(textLine -> Arrays.asList(textLine.toLowerCase().split("\W+"))) + .groupBy((key, word) -> word) + .count(Materialized.>as("counts-store")); + wordCounts.toStream().to("WordsWithCountsTopic", Produced.with(Serdes.String(), Serdes.Long())); + + KafkaStreams streams = new KafkaStreams(builder.build(), props); + streams.start(); + } + +} + + +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KTable; +import org.apache.kafka.streams.kstream.ValueMapper; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.kstream.Materialized; +import org.apache.kafka.streams.kstream.Produced; +import org.apache.kafka.streams.state.KeyValueStore; + +import java.util.Arrays; +import java.util.Properties; + +public class WordCountApplication { + + public static void main(final String[] args) throws Exception { + Properties props = new Properties(); + props.put(StreamsConfig.APPLICATION_ID_CONFIG, "wordcount-application"); + props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka-broker1:9092"); + props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass()); + props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); + + StreamsBuilder builder = new StreamsBuilder(); + KStream textLines = builder.stream("TextLinesTopic"); + KTable wordCounts = textLines + .flatMapValues(new ValueMapper>() { + @Override + public Iterable apply(String textLine) { + return Arrays.asList(textLine.toLowerCase().split("\W+")); + } + }) + .groupBy(new KeyValueMapper() { + @Override + public String apply(String key, String word) { + return word; + } + }) + .count(Materialized.>as("counts-store")); + + + wordCounts.toStream().to("WordsWithCountsTopic", Produced.with(Serdes.String(), Serdes.Long())); + + KafkaStreams streams = new KafkaStreams(builder.build(), props); + streams.start(); + } + +} +``` +{{% /tab %}} +{{% tab header="Scala" %}} +```scala +import java.util.Properties +import java.util.concurrent.TimeUnit + +import org.apache.kafka.streams.kstream.Materialized +import org.apache.kafka.streams.scala.ImplicitConversions._ +import org.apache.kafka.streams.scala._ +import org.apache.kafka.streams.scala.kstream._ +import org.apache.kafka.streams.{KafkaStreams, StreamsConfig} + +object WordCountApplication extends App { + import Serdes._ + + val props: Properties = { + val p = new Properties() + p.put(StreamsConfig.APPLICATION_ID_CONFIG, "wordcount-application") + p.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka-broker1:9092") + p + } + + val builder: StreamsBuilder = new StreamsBuilder + val textLines: KStream[String, String] = builder.stream[String, String]("TextLinesTopic") + val wordCounts: KTable[String, Long] = textLines + .flatMapValues(textLine => textLine.toLowerCase.split("\W+")) + .groupBy((_, word) => word) + .count()(Materialized.as("counts-store")) + wordCounts.toStream.to("WordsWithCountsTopic") + + val streams: KafkaStreams = new KafkaStreams(builder.build(), props) + streams.start() + + sys.ShutdownHookThread { + streams.close(10, TimeUnit.SECONDS) + } +} +``` +{{% /tab %}} +{{< /tabpane >}} + diff --git a/docs/streams/quickstart.html b/docs/streams/quickstart.html deleted file mode 100644 index 5bb106d77a206..0000000000000 --- a/docs/streams/quickstart.html +++ /dev/null @@ -1,381 +0,0 @@ - - - - - -
- - - -
- - -
- - diff --git a/docs/streams/quickstart.md b/docs/streams/quickstart.md new file mode 100644 index 0000000000000..63c26c29620d1 --- /dev/null +++ b/docs/streams/quickstart.md @@ -0,0 +1,258 @@ +--- +title: Quick Start +description: +weight: 2 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +# Run Kafka Streams Demo Application + +This tutorial assumes you are starting fresh and have no existing Kafka or ZooKeeper data. However, if you have already started Kafka, feel free to skip the first two steps. + +Kafka Streams is a client library for building mission-critical real-time applications and microservices, where the input and/or output data is stored in Kafka clusters. Kafka Streams combines the simplicity of writing and deploying standard Java and Scala applications on the client side with the benefits of Kafka's server-side cluster technology to make these applications highly scalable, elastic, fault-tolerant, distributed, and much more. + +This quickstart example will demonstrate how to run a streaming application coded in this library. Here is the gist of the `[WordCountDemo](https://github.com/apache/kafka/blob/3.7/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java)` example code (converted to use Java 8 lambda expressions for easy reading). + + + // Serializers/deserializers (serde) for String and Long types + final Serde stringSerde = Serdes.String(); + final Serde longSerde = Serdes.Long(); + + // Construct a `KStream` from the input topic "streams-plaintext-input", where message values + // represent lines of text (for the sake of this example, we ignore whatever may be stored + // in the message keys). + KStream textLines = builder.stream( + "streams-plaintext-input", + Consumed.with(stringSerde, stringSerde) + ); + + KTable wordCounts = textLines + // Split each text line, by whitespace, into words. + .flatMapValues(value -> Arrays.asList(value.toLowerCase().split("\W+"))) + + // Group the text words as message keys + .groupBy((key, value) -> value) + + // Count the occurrences of each word (message key). + .count(); + + // Store the running counts as a changelog stream to the output topic. + wordCounts.toStream().to("streams-wordcount-output", Produced.with(Serdes.String(), Serdes.Long())); + +It implements the WordCount algorithm, which computes a word occurrence histogram from the input text. However, unlike other WordCount examples you might have seen before that operate on bounded data, the WordCount demo application behaves slightly differently because it is designed to operate on an **infinite, unbounded stream** of data. Similar to the bounded variant, it is a stateful algorithm that tracks and updates the counts of words. However, since it must assume potentially unbounded input data, it will periodically output its current state and results while continuing to process more data because it cannot know when it has processed "all" the input data. + +As the first step, we will start Kafka (unless you already have it started) and then we will prepare input data to a Kafka topic, which will subsequently be processed by a Kafka Streams application. + +### Step 1: Download the code + +[Download](https://www.apache.org/dyn/closer.cgi?path=/kafka/3.7.2/kafka_2.13-3.7.2.tgz "Kafka downloads") the 3.7.2 release and un-tar it. Note that there are multiple downloadable Scala versions and we choose to use the recommended version (2.13) here: + + + > tar -xzf kafka_2.13-3.7.2.tgz + > cd kafka_2.13-3.7.2 + +### Step 2: Start the Kafka server + +Apache Kafka can be started using ZooKeeper or KRaft. To get started with either configuration follow one of the sections below but not both. + +#### Kafka with ZooKeeper + +Run the following commands in order to start all services in the correct order: + + + > bin/zookeeper-server-start.sh config/zookeeper.properties + +Open another terminal session and run: + + + > bin/kafka-server-start.sh config/server.properties + +#### Kafka with KRaft + +Generate a Cluster UUID + + + > KAFKA_CLUSTER_ID="$(bin/kafka-storage.sh random-uuid)" + +Format Log Directories + + + > bin/kafka-storage.sh format -t $KAFKA_CLUSTER_ID -c config/kraft/server.properties + +Start the Kafka Server + + + > bin/kafka-server-start.sh config/kraft/server.properties + +### Step 3: Prepare input topic and start Kafka producer + +Next, we create the input topic named **streams-plaintext-input** and the output topic named **streams-wordcount-output** : + + + > bin/kafka-topics.sh --create \ + --bootstrap-server localhost:9092 \ + --replication-factor 1 \ + --partitions 1 \ + --topic streams-plaintext-input + Created topic "streams-plaintext-input". + +Note: we create the output topic with compaction enabled because the output stream is a changelog stream (cf. explanation of application output below). + + + > bin/kafka-topics.sh --create \ + --bootstrap-server localhost:9092 \ + --replication-factor 1 \ + --partitions 1 \ + --topic streams-wordcount-output \ + --config cleanup.policy=compact + Created topic "streams-wordcount-output". + +The created topic can be described with the same **kafka-topics** tool: + + + > bin/kafka-topics.sh --bootstrap-server localhost:9092 --describe + + Topic:streams-wordcount-output PartitionCount:1 ReplicationFactor:1 Configs:cleanup.policy=compact,segment.bytes=1073741824 + Topic: streams-wordcount-output Partition: 0 Leader: 0 Replicas: 0 Isr: 0 + Topic:streams-plaintext-input PartitionCount:1 ReplicationFactor:1 Configs:segment.bytes=1073741824 + Topic: streams-plaintext-input Partition: 0 Leader: 0 Replicas: 0 Isr: 0 + +### Step 4: Start the Wordcount Application + +The following command starts the WordCount demo application: + + + > bin/kafka-run-class.sh org.apache.kafka.streams.examples.wordcount.WordCountDemo + +The demo application will read from the input topic **streams-plaintext-input** , perform the computations of the WordCount algorithm on each of the read messages, and continuously write its current results to the output topic **streams-wordcount-output**. Hence there won't be any STDOUT output except log entries as the results are written back into in Kafka. + +Now we can start the console producer in a separate terminal to write some input data to this topic: + + + > bin/kafka-console-producer.sh --bootstrap-server localhost:9092 --topic streams-plaintext-input + +and inspect the output of the WordCount demo application by reading from its output topic with the console consumer in a separate terminal: + + + > bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 \ + --topic streams-wordcount-output \ + --from-beginning \ + --formatter kafka.tools.DefaultMessageFormatter \ + --property print.key=true \ + --property print.value=true \ + --property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \ + --property value.deserializer=org.apache.kafka.common.serialization.LongDeserializer + +### Step 5: Process some data + +Now let's write some message with the console producer into the input topic **streams-plaintext-input** by entering a single line of text and then hit . This will send a new message to the input topic, where the message key is null and the message value is the string encoded text line that you just entered (in practice, input data for applications will typically be streaming continuously into Kafka, rather than being manually entered as we do in this quickstart): + + + > bin/kafka-console-producer.sh --bootstrap-server localhost:9092 --topic streams-plaintext-input + all streams lead to kafka + +This message will be processed by the Wordcount application and the following output data will be written to the **streams-wordcount-output** topic and printed by the console consumer: + + + > bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 \ + --topic streams-wordcount-output \ + --from-beginning \ + --formatter kafka.tools.DefaultMessageFormatter \ + --property print.key=true \ + --property print.value=true \ + --property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \ + --property value.deserializer=org.apache.kafka.common.serialization.LongDeserializer + + all 1 + streams 1 + lead 1 + to 1 + kafka 1 + +Here, the first column is the Kafka message key in `java.lang.String` format and represents a word that is being counted, and the second column is the message value in `java.lang.Long`format, representing the word's latest count. + +Now let's continue writing one more message with the console producer into the input topic **streams-plaintext-input**. Enter the text line "hello kafka streams" and hit . Your terminal should look as follows: + + + > bin/kafka-console-producer.sh --bootstrap-server localhost:9092 --topic streams-plaintext-input + all streams lead to kafka + hello kafka streams + +In your other terminal in which the console consumer is running, you will observe that the WordCount application wrote new output data: + + + > bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 \ + --topic streams-wordcount-output \ + --from-beginning \ + --formatter kafka.tools.DefaultMessageFormatter \ + --property print.key=true \ + --property print.value=true \ + --property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \ + --property value.deserializer=org.apache.kafka.common.serialization.LongDeserializer + + all 1 + streams 1 + lead 1 + to 1 + kafka 1 + hello 1 + kafka 2 + streams 2 + +Here the last printed lines **kafka 2** and **streams 2** indicate updates to the keys **kafka** and **streams** whose counts have been incremented from **1** to **2**. Whenever you write further input messages to the input topic, you will observe new messages being added to the **streams-wordcount-output** topic, representing the most recent word counts as computed by the WordCount application. Let's enter one final input text line "join kafka summit" and hit in the console producer to the input topic **streams-plaintext-input** before we wrap up this quickstart: + + + > bin/kafka-console-producer.sh --bootstrap-server localhost:9092 --topic streams-plaintext-input + all streams lead to kafka + hello kafka streams + join kafka summit + +The **streams-wordcount-output** topic will subsequently show the corresponding updated word counts (see last three lines): + + + > bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 \ + --topic streams-wordcount-output \ + --from-beginning \ + --formatter kafka.tools.DefaultMessageFormatter \ + --property print.key=true \ + --property print.value=true \ + --property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \ + --property value.deserializer=org.apache.kafka.common.serialization.LongDeserializer + + all 1 + streams 1 + lead 1 + to 1 + kafka 1 + hello 1 + kafka 2 + streams 2 + join 1 + kafka 3 + summit 1 + +As one can see, outputs of the Wordcount application is actually a continuous stream of updates, where each output record (i.e. each line in the original output above) is an updated count of a single word, aka record key such as "kafka". For multiple records with the same key, each later record is an update of the previous one. + +The two diagrams below illustrate what is essentially happening behind the scenes. The first column shows the evolution of the current state of the `KTable` that is counting word occurrences for `count`. The second column shows the change records that result from state updates to the KTable and that are being sent to the output Kafka topic **streams-wordcount-output**. + +![](/37/images/streams-table-updates-02.png) ![](/37/images/streams-table-updates-01.png) + +First the text line "all streams lead to kafka" is being processed. The `KTable` is being built up as each new word results in a new table entry (highlighted with a green background), and a corresponding change record is sent to the downstream `KStream`. + +When the second text line "hello kafka streams" is processed, we observe, for the first time, that existing entries in the `KTable` are being updated (here: for the words "kafka" and for "streams"). And again, change records are being sent to the output topic. + +And so on (we skip the illustration of how the third line is being processed). This explains why the output topic has the contents we showed above, because it contains the full record of changes. + +Looking beyond the scope of this concrete example, what Kafka Streams is doing here is to leverage the duality between a table and a changelog stream (here: table = the KTable, changelog stream = the downstream KStream): you can publish every change of the table to a stream, and if you consume the entire changelog stream from beginning to end, you can reconstruct the contents of the table. + +### Step 6: Teardown the application + +You can now stop the console consumer, the console producer, the Wordcount application, the Kafka broker and the ZooKeeper server (if one was started) in order via **Ctrl-C**. + + * [Documentation](/documentation) + * [Kafka Streams](/streams) + + diff --git a/docs/streams/tutorial.html b/docs/streams/tutorial.html deleted file mode 100644 index ac7c4b10c866d..0000000000000 --- a/docs/streams/tutorial.html +++ /dev/null @@ -1,608 +0,0 @@ - - - - - -
- - - -
- - -
- - diff --git a/docs/streams/tutorial.md b/docs/streams/tutorial.md new file mode 100644 index 0000000000000..f10825a6a7c44 --- /dev/null +++ b/docs/streams/tutorial.md @@ -0,0 +1,446 @@ +--- +title: Write a streams app +description: +weight: 3 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +# Tutorial: Write a Kafka Streams Application + +In this guide we will start from scratch on setting up your own project to write a stream processing application using Kafka Streams. It is highly recommended to read the [quickstart](/37/streams/quickstart) first on how to run a Streams application written in Kafka Streams if you have not done so. + +### Setting up a Maven Project + +We are going to use a Kafka Streams Maven Archetype for creating a Streams project structure with the following commands: + + + mvn archetype:generate \ + -DarchetypeGroupId=org.apache.kafka \ + -DarchetypeArtifactId=streams-quickstart-java \ + -DarchetypeVersion=3.7.2 \ + -DgroupId=streams.examples \ + -DartifactId=streams-quickstart \ + -Dversion=0.1 \ + -Dpackage=myapps + +You can use a different value for `groupId`, `artifactId` and `package` parameters if you like. Assuming the above parameter values are used, this command will create a project structure that looks like this: + + + > tree streams-quickstart + streams-quickstart + |-- pom.xml + |-- src + |-- main + |-- java + | |-- myapps + | |-- LineSplit.java + | |-- Pipe.java + | |-- WordCount.java + |-- resources + |-- log4j.properties + +The `pom.xml` file included in the project already has the Streams dependency defined. Note, that the generated `pom.xml` targets Java 8, and does not work with higher Java versions. + +There are already several example programs written with Streams library under `src/main/java`. Since we are going to start writing such programs from scratch, we can now delete these examples: + + + > cd streams-quickstart + > rm src/main/java/myapps/*.java + +### Writing a first Streams application: Pipe + +It's coding time now! Feel free to open your favorite IDE and import this Maven project, or simply open a text editor and create a java file under `src/main/java/myapps`. Let's name it `Pipe.java`: + + + package myapps; + + public class Pipe { + + public static void main(String[] args) throws Exception { + + } + } + +We are going to fill in the `main` function to write this pipe program. Note that we will not list the import statements as we go since IDEs can usually add them automatically. However if you are using a text editor you need to manually add the imports, and at the end of this section we'll show the complete code snippet with import statement for you. + +The first step to write a Streams application is to create a `java.util.Properties` map to specify different Streams execution configuration values as defined in `StreamsConfig`. A couple of important configuration values you need to set are: `StreamsConfig.BOOTSTRAP_SERVERS_CONFIG`, which specifies a list of host/port pairs to use for establishing the initial connection to the Kafka cluster, and `StreamsConfig.APPLICATION_ID_CONFIG`, which gives the unique identifier of your Streams application to distinguish itself with other applications talking to the same Kafka cluster: + + + Properties props = new Properties(); + props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-pipe"); + props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); // assuming that the Kafka broker this application is talking to runs on local machine with port 9092 + +In addition, you can customize other configurations in the same map, for example, default serialization and deserialization libraries for the record key-value pairs: + + + props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass()); + props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); + +For a full list of configurations of Kafka Streams please refer to this [table](/37/#streamsconfigs). + +Next we will define the computational logic of our Streams application. In Kafka Streams this computational logic is defined as a `topology` of connected processor nodes. We can use a topology builder to construct such a topology, + + + final StreamsBuilder builder = new StreamsBuilder(); + +And then create a source stream from a Kafka topic named `streams-plaintext-input` using this topology builder: + + + KStream source = builder.stream("streams-plaintext-input"); + +Now we get a `KStream` that is continuously generating records from its source Kafka topic `streams-plaintext-input`. The records are organized as `String` typed key-value pairs. The simplest thing we can do with this stream is to write it into another Kafka topic, say it's named `streams-pipe-output`: + + + source.to("streams-pipe-output"); + +Note that we can also concatenate the above two lines into a single line as: + + + builder.stream("streams-plaintext-input").to("streams-pipe-output"); + +We can inspect what kind of `topology` is created from this builder by doing the following: + + + final Topology topology = builder.build(); + +And print its description to standard output as: + + + System.out.println(topology.describe()); + +If we just stop here, compile and run the program, it will output the following information: + + + > mvn clean package + > mvn exec:java -Dexec.mainClass=myapps.Pipe + Sub-topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000(topics: streams-plaintext-input) --> KSTREAM-SINK-0000000001 + Sink: KSTREAM-SINK-0000000001(topic: streams-pipe-output) <-- KSTREAM-SOURCE-0000000000 + Global Stores: + none + +As shown above, it illustrates that the constructed topology has two processor nodes, a source node `KSTREAM-SOURCE-0000000000` and a sink node `KSTREAM-SINK-0000000001`. `KSTREAM-SOURCE-0000000000` continuously read records from Kafka topic `streams-plaintext-input` and pipe them to its downstream node `KSTREAM-SINK-0000000001`; `KSTREAM-SINK-0000000001` will write each of its received record in order to another Kafka topic `streams-pipe-output` (the `-->` and `<--` arrows dictates the downstream and upstream processor nodes of this node, i.e. "children" and "parents" within the topology graph). It also illustrates that this simple topology has no global state stores associated with it (we will talk about state stores more in the following sections). + +Note that we can always describe the topology as we did above at any given point while we are building it in the code, so as a user you can interactively "try and taste" your computational logic defined in the topology until you are happy with it. Suppose we are already done with this simple topology that just pipes data from one Kafka topic to another in an endless streaming manner, we can now construct the Streams client with the two components we have just constructed above: the configuration map specified in a `java.util.Properties` instance and the `Topology` object. + + + final KafkaStreams streams = new KafkaStreams(topology, props); + +By calling its `start()` function we can trigger the execution of this client. The execution won't stop until `close()` is called on this client. We can, for example, add a shutdown hook with a countdown latch to capture a user interrupt and close the client upon terminating this program: + + + final CountDownLatch latch = new CountDownLatch(1); + + // attach shutdown handler to catch control-c + Runtime.getRuntime().addShutdownHook(new Thread("streams-shutdown-hook") { + @Override + public void run() { + streams.close(); + latch.countDown(); + } + }); + + try { + streams.start(); + latch.await(); + } catch (Throwable e) { + System.exit(1); + } + System.exit(0); + +The complete code so far looks like this: + + + package myapps; + + import org.apache.kafka.common.serialization.Serdes; + import org.apache.kafka.streams.KafkaStreams; + import org.apache.kafka.streams.StreamsBuilder; + import org.apache.kafka.streams.StreamsConfig; + import org.apache.kafka.streams.Topology; + + import java.util.Properties; + import java.util.concurrent.CountDownLatch; + + public class Pipe { + + public static void main(String[] args) throws Exception { + Properties props = new Properties(); + props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-pipe"); + props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); + props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass()); + props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); + + final StreamsBuilder builder = new StreamsBuilder(); + + builder.stream("streams-plaintext-input").to("streams-pipe-output"); + + final Topology topology = builder.build(); + + final KafkaStreams streams = new KafkaStreams(topology, props); + final CountDownLatch latch = new CountDownLatch(1); + + // attach shutdown handler to catch control-c + Runtime.getRuntime().addShutdownHook(new Thread("streams-shutdown-hook") { + @Override + public void run() { + streams.close(); + latch.countDown(); + } + }); + + try { + streams.start(); + latch.await(); + } catch (Throwable e) { + System.exit(1); + } + System.exit(0); + } + } + +If you already have the Kafka broker up and running at `localhost:9092`, and the topics `streams-plaintext-input` and `streams-pipe-output` created on that broker, you can run this code in your IDE or on the command line, using Maven: + + + > mvn clean package + > mvn exec:java -Dexec.mainClass=myapps.Pipe + +For detailed instructions on how to run a Streams application and observe its computing results, please read the [Play with a Streams Application](/37/streams/quickstart) section. We will not talk about this in the rest of this section. + +### Writing a second Streams application: Line Split + +We have learned how to construct a Streams client with its two key components: the `StreamsConfig` and `Topology`. Now let's move on to add some real processing logic by augmenting the current topology. We can first create another program by first copy the existing `Pipe.java` class: + + + > cp src/main/java/myapps/Pipe.java src/main/java/myapps/LineSplit.java + +And change its class name as well as the application id config to distinguish with the original program: + + + public class LineSplit { + + public static void main(String[] args) throws Exception { + Properties props = new Properties(); + props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-linesplit"); + // ... + } + } + +Since each of the source stream's record is a `String` typed key-value pair, let's treat the value string as a text line and split it into words with a `FlatMapValues` operator: + + + KStream source = builder.stream("streams-plaintext-input"); + KStream words = source.flatMapValues(new ValueMapper>() { + @Override + public Iterable apply(String value) { + return Arrays.asList(value.split("\W+")); + } + }); + +The operator will take the `source` stream as its input, and generate a new stream named `words` by processing each record from its source stream in order and breaking its value string into a list of words, and producing each word as a new record to the output `words` stream. This is a stateless operator that does not need to keep track of any previously received records or processed results. Note if you are using JDK 8 you can use lambda expression and simplify the above code as: + + + KStream source = builder.stream("streams-plaintext-input"); + KStream words = source.flatMapValues(value -> Arrays.asList(value.split("\W+"))); + +And finally we can write the word stream back into another Kafka topic, say `streams-linesplit-output`. Again, these two steps can be concatenated as the following (assuming lambda expression is used): + + + KStream source = builder.stream("streams-plaintext-input"); + source.flatMapValues(value -> Arrays.asList(value.split("\W+"))) + .to("streams-linesplit-output"); + +If we now describe this augmented topology as `System.out.println(topology.describe())`, we will get the following: + + + > mvn clean package + > mvn exec:java -Dexec.mainClass=myapps.LineSplit + Sub-topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000(topics: streams-plaintext-input) --> KSTREAM-FLATMAPVALUES-0000000001 + Processor: KSTREAM-FLATMAPVALUES-0000000001(stores: []) --> KSTREAM-SINK-0000000002 <-- KSTREAM-SOURCE-0000000000 + Sink: KSTREAM-SINK-0000000002(topic: streams-linesplit-output) <-- KSTREAM-FLATMAPVALUES-0000000001 + Global Stores: + none + +As we can see above, a new processor node `KSTREAM-FLATMAPVALUES-0000000001` is injected into the topology between the original source and sink nodes. It takes the source node as its parent and the sink node as its child. In other words, each record fetched by the source node will first traverse to the newly added `KSTREAM-FLATMAPVALUES-0000000001` node to be processed, and one or more new records will be generated as a result. They will continue traverse down to the sink node to be written back to Kafka. Note this processor node is "stateless" as it is not associated with any stores (i.e. `(stores: [])`). + +The complete code looks like this (assuming lambda expression is used): + + + package myapps; + + import org.apache.kafka.common.serialization.Serdes; + import org.apache.kafka.streams.KafkaStreams; + import org.apache.kafka.streams.StreamsBuilder; + import org.apache.kafka.streams.StreamsConfig; + import org.apache.kafka.streams.Topology; + import org.apache.kafka.streams.kstream.KStream; + + import java.util.Arrays; + import java.util.Properties; + import java.util.concurrent.CountDownLatch; + + public class LineSplit { + + public static void main(String[] args) throws Exception { + Properties props = new Properties(); + props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-linesplit"); + props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); + props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass()); + props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); + + final StreamsBuilder builder = new StreamsBuilder(); + + KStream source = builder.stream("streams-plaintext-input"); + source.flatMapValues(value -> Arrays.asList(value.split("\W+"))) + .to("streams-linesplit-output"); + + final Topology topology = builder.build(); + final KafkaStreams streams = new KafkaStreams(topology, props); + final CountDownLatch latch = new CountDownLatch(1); + + // ... same as Pipe.java above + } + } + +### Writing a third Streams application: Wordcount + +Let's now take a step further to add some "stateful" computations to the topology by counting the occurrence of the words split from the source text stream. Following similar steps let's create another program based on the `LineSplit.java` class: + + + public class WordCount { + + public static void main(String[] args) throws Exception { + Properties props = new Properties(); + props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-wordcount"); + // ... + } + } + +In order to count the words we can first modify the `flatMapValues` operator to treat all of them as lower case (assuming lambda expression is used): + + + source.flatMapValues(new ValueMapper>() { + @Override + public Iterable apply(String value) { + return Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\W+")); + } + }); + +In order to do the counting aggregation we have to first specify that we want to key the stream on the value string, i.e. the lower cased word, with a `groupBy` operator. This operator generate a new grouped stream, which can then be aggregated by a `count` operator, which generates a running count on each of the grouped keys: + + + KTable counts = + source.flatMapValues(new ValueMapper>() { + @Override + public Iterable apply(String value) { + return Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\W+")); + } + }) + .groupBy(new KeyValueMapper() { + @Override + public String apply(String key, String value) { + return value; + } + }) + // Materialize the result into a KeyValueStore named "counts-store". + // The Materialized store is always of type as this is the format of the inner most store. + .count(Materialized.> as("counts-store")); + +Note that the `count` operator has a `Materialized` parameter that specifies that the running count should be stored in a state store named `counts-store`. This `counts-store` store can be queried in real-time, with details described in the [Developer Manual](/37/streams/developer-guide#streams_interactive_queries). + +We can also write the `counts` KTable's changelog stream back into another Kafka topic, say `streams-wordcount-output`. Because the result is a changelog stream, the output topic `streams-wordcount-output` should be configured with log compaction enabled. Note that this time the value type is no longer `String` but `Long`, so the default serialization classes are not viable for writing it to Kafka anymore. We need to provide overridden serialization methods for `Long` types, otherwise a runtime exception will be thrown: + + + counts.toStream().to("streams-wordcount-output", Produced.with(Serdes.String(), Serdes.Long())); + +Note that in order to read the changelog stream from topic `streams-wordcount-output`, one needs to set the value deserialization as `org.apache.kafka.common.serialization.LongDeserializer`. Details of this can be found in the [Play with a Streams Application](/37/streams/quickstart) section. Assuming lambda expression from JDK 8 can be used, the above code can be simplified as: + + + KStream source = builder.stream("streams-plaintext-input"); + source.flatMapValues(value -> Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\W+"))) + .groupBy((key, value) -> value) + .count(Materialized.>as("counts-store")) + .toStream() + .to("streams-wordcount-output", Produced.with(Serdes.String(), Serdes.Long())); + +If we again describe this augmented topology as `System.out.println(topology.describe())`, we will get the following: + + + > mvn clean package + > mvn exec:java -Dexec.mainClass=myapps.WordCount + Sub-topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000(topics: streams-plaintext-input) --> KSTREAM-FLATMAPVALUES-0000000001 + Processor: KSTREAM-FLATMAPVALUES-0000000001(stores: []) --> KSTREAM-KEY-SELECT-0000000002 <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-KEY-SELECT-0000000002(stores: []) --> KSTREAM-FILTER-0000000005 <-- KSTREAM-FLATMAPVALUES-0000000001 + Processor: KSTREAM-FILTER-0000000005(stores: []) --> KSTREAM-SINK-0000000004 <-- KSTREAM-KEY-SELECT-0000000002 + Sink: KSTREAM-SINK-0000000004(topic: counts-store-repartition) <-- KSTREAM-FILTER-0000000005 + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000006(topics: counts-store-repartition) --> KSTREAM-AGGREGATE-0000000003 + Processor: KSTREAM-AGGREGATE-0000000003(stores: [counts-store]) --> KTABLE-TOSTREAM-0000000007 <-- KSTREAM-SOURCE-0000000006 + Processor: KTABLE-TOSTREAM-0000000007(stores: []) --> KSTREAM-SINK-0000000008 <-- KSTREAM-AGGREGATE-0000000003 + Sink: KSTREAM-SINK-0000000008(topic: streams-wordcount-output) <-- KTABLE-TOSTREAM-0000000007 + Global Stores: + none + +As we can see above, the topology now contains two disconnected sub-topologies. The first sub-topology's sink node `KSTREAM-SINK-0000000004` will write to a repartition topic `counts-store-repartition`, which will be read by the second sub-topology's source node `KSTREAM-SOURCE-0000000006`. The repartition topic is used to "shuffle" the source stream by its aggregation key, which is in this case the value string. In addition, inside the first sub-topology a stateless `KSTREAM-FILTER-0000000005` node is injected between the grouping `KSTREAM-KEY-SELECT-0000000002` node and the sink node to filter out any intermediate record whose aggregate key is empty. + +In the second sub-topology, the aggregation node `KSTREAM-AGGREGATE-0000000003` is associated with a state store named `counts-store` (the name is specified by the user in the `count` operator). Upon receiving each record from its upcoming stream source node, the aggregation processor will first query its associated `counts-store` store to get the current count for that key, augment by one, and then write the new count back to the store. Each updated count for the key will also be piped downstream to the `KTABLE-TOSTREAM-0000000007` node, which interpret this update stream as a record stream before further piping to the sink node `KSTREAM-SINK-0000000008` for writing back to Kafka. + +The complete code looks like this (assuming lambda expression is used): + + + package myapps; + + import org.apache.kafka.common.serialization.Serdes; + import org.apache.kafka.common.utils.Bytes; + import org.apache.kafka.streams.KafkaStreams; + import org.apache.kafka.streams.StreamsBuilder; + import org.apache.kafka.streams.StreamsConfig; + import org.apache.kafka.streams.Topology; + import org.apache.kafka.streams.kstream.KStream; + import org.apache.kafka.streams.kstream.Materialized; + import org.apache.kafka.streams.kstream.Produced; + import org.apache.kafka.streams.state.KeyValueStore; + + import java.util.Arrays; + import java.util.Locale; + import java.util.Properties; + import java.util.concurrent.CountDownLatch; + + public class WordCount { + + public static void main(String[] args) throws Exception { + Properties props = new Properties(); + props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-wordcount"); + props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); + props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass()); + props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); + + final StreamsBuilder builder = new StreamsBuilder(); + + KStream source = builder.stream("streams-plaintext-input"); + source.flatMapValues(value -> Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\W+"))) + .groupBy((key, value) -> value) + .count(Materialized.>as("counts-store")) + .toStream() + .to("streams-wordcount-output", Produced.with(Serdes.String(), Serdes.Long())); + + final Topology topology = builder.build(); + final KafkaStreams streams = new KafkaStreams(topology, props); + final CountDownLatch latch = new CountDownLatch(1); + + // ... same as Pipe.java above + } + } + + * [Documentation](/documentation) + * [Kafka Streams](/streams) + + diff --git a/docs/streams/upgrade-guide.html b/docs/streams/upgrade-guide.html deleted file mode 100644 index 09328389c434f..0000000000000 --- a/docs/streams/upgrade-guide.html +++ /dev/null @@ -1,1631 +0,0 @@ - - - - - - - - -
- - -
- - diff --git a/docs/streams/upgrade-guide.md b/docs/streams/upgrade-guide.md new file mode 100644 index 0000000000000..723076acdf146 --- /dev/null +++ b/docs/streams/upgrade-guide.md @@ -0,0 +1,795 @@ +--- +title: Upgrade Guide +description: +weight: 6 +tags: ['kafka', 'docs'] +aliases: +keywords: +type: docs +--- + +# Upgrade Guide and API Changes + +Upgrading from any older version to 3.7.2 is possible: if upgrading from 3.4 or below, you will need to do two rolling bounces, where during the first rolling bounce phase you set the config `upgrade.from="older version"` (possible values are `"0.10.0" - "3.4"`) and during the second you remove it. This is required to safely handle 3 changes. The first is introduction of the new cooperative rebalancing protocol of the embedded consumer. The second is a change in foreign-key join serialization format. Note that you will remain using the old eager rebalancing protocol if you skip or delay the second rolling bounce, but you can safely switch over to cooperative at any time once the entire group is on 2.4+ by removing the config value and bouncing. For more details please refer to [KIP-429](https://cwiki.apache.org/confluence/x/vAclBg). The third is a change in the serialization format for an internal repartition topic. For more details, please refer to [KIP-904](https://cwiki.apache.org/confluence/x/P5VbDg): + + * prepare your application instances for a rolling bounce and make sure that config `upgrade.from` is set to the version from which it is being upgrade. + * bounce each instance of your application once + * prepare your newly deployed 3.7.2 application instances for a second round of rolling bounces; make sure to remove the value for config `upgrade.from` + * bounce each instance of your application once more to complete the upgrade + + + +As an alternative, an offline upgrade is also possible. Upgrading from any versions as old as 0.10.0.x to 3.7.2 in offline mode require the following steps: + + * stop all old (e.g., 0.10.0.x) application instances + * update your code and swap old code and jar file with new code and new jar file + * restart all new (3.7.2) application instances + + + +Note: The cooperative rebalancing protocol has been the default since 2.4, but we have continued to support the eager rebalancing protocol to provide users an upgrade path. This support will be dropped in a future release, so any users still on the eager protocol should prepare to finish upgrading their applications to the cooperative protocol in version 3.1. This only affects users who are still on a version older than 2.4, and users who have upgraded already but have not yet removed the `upgrade.from` config that they set when upgrading from a version below 2.4. Users fitting into the latter case will simply need to unset this config when upgrading beyond 3.1, while users in the former case will need to follow a slightly different upgrade path if they attempt to upgrade from 2.3 or below to a version above 3.1. Those applications will need to go through a bridge release, by first upgrading to a version between 2.4 - 3.1 and setting the `upgrade.from` config, then removing that config and upgrading to the final version above 3.1. See [KAFKA-8575](https://issues.apache.org/jira/browse/KAFKA-8575) for more details. + +For a table that shows Streams API compatibility with Kafka broker versions, see Broker Compatibility. + +## Notable compatibility changes in past releases + +Downgrading from 3.5.x or newer version to 3.4.x or older version needs special attention: Since 3.5.0 release, Kafka Streams uses a new serialization format for repartition topics. This means that older versions of Kafka Streams would not be able to recognize the bytes written by newer versions, and hence it is harder to downgrade Kafka Streams with version 3.5.0 or newer to older versions in-flight. For more details, please refer to [KIP-904](https://cwiki.apache.org/confluence/x/P5VbDg). For a downgrade, first switch the config from `"upgrade.from"` to the version you are downgrading to. This disables writing of the new serialization format in your application. It's important to wait in this state long enough to make sure that the application has finished processing any "in-flight" messages written into the repartition topics in the new serialization format. Afterwards, you can downgrade your application to a pre-3.5.x version. + +Downgrading from 3.0.x or newer version to 2.8.x or older version needs special attention: Since 3.0.0 release, Kafka Streams uses a newer RocksDB version whose on-disk format changed. This means that old versioned RocksDB would not be able to recognize the bytes written by that newer versioned RocksDB, and hence it is harder to downgrade Kafka Streams with version 3.0.0 or newer to older versions in-flight. Users need to wipe out the local RocksDB state stores written by the new versioned Kafka Streams before swapping in the older versioned Kafka Streams bytecode, which would then restore the state stores with the old on-disk format from the changelogs. + +Kafka Streams does not support running multiple instances of the same application as different processes on the same physical state directory. Starting in 2.8.0 (as well as 2.7.1 and 2.6.2), this restriction will be enforced. If you wish to run more than one instance of Kafka Streams, you must configure them with different values for `state.dir`. + +Starting in Kafka Streams 2.6.x, a new processing mode is available, named EOS version 2. This can be configured by setting `"processing.guarantee"` to `"exactly_once_v2"` for application versions 3.0+, or setting it to `"exactly_once_beta"` for versions between 2.6 and 2.8. To use this new feature, your brokers must be on version 2.5.x or newer. If you want to upgrade your EOS application from an older version and enable this feature in version 3.0+, you first need to upgrade your application to version 3.0.x, staying on `"exactly_once"`, and then do second round of rolling bounces to switch to `"exactly_once_v2"`. If you are upgrading an EOS application from an older (pre-2.6) version to a version between 2.6 and 2.8, follow these same steps but with the config `"exactly_once_beta"` instead. No special steps are required to upgrade an application using `"exactly_once_beta"` from version 2.6+ to 3.0 or higher: you can just change the config from `"exactly_once_beta"` to `"exactly_once_v2"` during the rolling upgrade. For a downgrade, do the reverse: first switch the config from `"exactly_once_v2"` to `"exactly_once"` to disable the feature in your 2.6.x application. Afterward, you can downgrade your application to a pre-2.6.x version. + +Since 2.6.0 release, Kafka Streams depends on a RocksDB version that requires MacOS 10.14 or higher. + +To run a Kafka Streams application version 2.2.1, 2.3.0, or higher a broker version 0.11.0 or higher is required and the on-disk message format must be 0.11 or higher. Brokers must be on version 0.10.1 or higher to run a Kafka Streams application version 0.10.1 to 2.2.0. Additionally, on-disk message format must be 0.10 or higher to run a Kafka Streams application version 1.0 to 2.2.0. For Kafka Streams 0.10.0, broker version 0.10.0 or higher is required. + +In deprecated `KStreamBuilder` class, when a `KTable` is created from a source topic via `KStreamBuilder.table()`, its materialized state store will reuse the source topic as its changelog topic for restoring, and will disable logging to avoid appending new updates to the source topic; in the `StreamsBuilder` class introduced in 1.0, this behavior was changed accidentally: we still reuse the source topic as the changelog topic for restoring, but will also create a separate changelog topic to append the update records from source topic to. In the 2.0 release, we have fixed this issue and now users can choose whether or not to reuse the source topic based on the `StreamsConfig#TOPOLOGY_OPTIMIZATION`: if you are upgrading from the old `KStreamBuilder` class and hence you need to change your code to use the new `StreamsBuilder`, you should set this config value to `StreamsConfig#OPTIMIZE` to continue reusing the source topic; if you are upgrading from 1.0 or 1.1 where you are already using `StreamsBuilder` and hence have already created a separate changelog topic, you should set this config value to `StreamsConfig#NO_OPTIMIZATION` when upgrading to 3.7.2 in order to use that changelog topic for restoring the state store. More details about the new config `StreamsConfig#TOPOLOGY_OPTIMIZATION` can be found in [KIP-295](https://cwiki.apache.org/confluence/display/KAFKA/KIP-295%3A+Add+Streams+Configuration+Allowing+for+Optional+Topology+Optimization). + +## Streams API changes in 3.7.0 + +We added a new method to `KafkaStreams`, namely `KafkaStreams#setStandbyUpdateListener()` in [KIP-988](https://cwiki.apache.org/confluence/display/KAFKA/KIP-988%3A+Streams+Standby+Update+Listener), in which users can provide their customized implementation of the newly added `StandbyUpdateListener` interface to continuously monitor changes to standby tasks. + +IQv2 supports `RangeQuery` that allows to specify unbounded, bounded, or half-open key-ranges, which return data in unordered (byte[]-lexicographical) order (per partition). [KIP-985](https://cwiki.apache.org/confluence/display/KAFKA/KIP-985%3A+Add+reverseRange+and+reverseAll+query+over+kv-store+in+IQv2) extends this functionality by adding `.withDescendingKeys()` and `.withAscendingKeys()`to allow user to receive data in descending or ascending order. + +[KIP-992](https://cwiki.apache.org/confluence/display/KAFKA/KIP-992%3A+Proposal+to+introduce+IQv2+Query+Types%3A+TimestampedKeyQuery+and+TimestampedRangeQuery) adds two new query types, namely `TimestampedKeyQuery` and `TimestampedRangeQuery`. Both should be used to query a timestamped key-value store, to retrieve a `ValueAndTimestamp` result. The existing `KeyQuery` and `RangeQuery` are changed to always return the value only for timestamped key-value stores. + +IQv2 adds support for `MultiVersionedKeyQuery` (introduced in [KIP-968](https://cwiki.apache.org/confluence/display/KAFKA/KIP-968%3A+Support+single-key_multi-timestamp+interactive+queries+%28IQv2%29+for+versioned+state+stores)) that allows retrieving a set of records from a versioned state store for a given key and a specified time range. Users have to use `fromTime(Instant)` and/or `toTime(Instant)` to specify a half or a complete time range. + +IQv2 adds support for `VersionedKeyQuery` (introduced in [KIP-960](https://cwiki.apache.org/confluence/display/KAFKA/KIP-960%3A+Support+single-key_single-timestamp+interactive+queries+%28IQv2%29+for+versioned+state+stores)) that allows retrieving a single record from a versioned state store based on its key and timestamp. Users have to use the `asOf(Instant)` method to define a query that returns the record's version for the specified timestamp. To be more precise, the key query returns the record with the greatest timestamp `<= Instant`. + +The non-null key requirements for Kafka Streams join operators were relaxed as part of [KIP-962](https://cwiki.apache.org/confluence/display/KAFKA/KIP-962%3A+Relax+non-null+key+requirement+in+Kafka+Streams). The behavior of the following operators changed. + + * left join KStream-KStream: no longer drop left records with null-key and call ValueJoiner with 'null' for right value. + * outer join KStream-KStream: no longer drop left/right records with null-key and call ValueJoiner with 'null' for right/left value. + * left-foreign-key join KTable-KTable: no longer drop left records with null-foreign-key returned by the ForeignKeyExtractor and call ValueJoiner with 'null' for right value. + * left join KStream-KTable: no longer drop left records with null-key and call ValueJoiner with 'null' for right value. + * left join KStream-GlobalTable: no longer drop records when KeyValueMapper returns 'null' and call ValueJoiner with 'null' for right value. + +Stream-DSL users who want to keep the current behavior can prepend a .filter() operator to the aforementioned operators and filter accordingly. The following snippets illustrate how to keep the old behavior. + + + + //left join KStream-KStream + leftStream + .filter((key, value) -> key != null) + .leftJoin(rightStream, (leftValue, rightValue) -> join(leftValue, rightValue), windows); + + //outer join KStream-KStream + rightStream + .filter((key, value) -> key != null); + leftStream + .filter((key, value) -> key != null) + .outerJoin(rightStream, (leftValue, rightValue) -> join(leftValue, rightValue), windows); + + //left-foreign-key join KTable-KTable + Function<String;, String> foreignKeyExtractor = leftValue -> ... + leftTable + .filter((key, value) -> foreignKeyExtractor.apply(value) != null) + .leftJoin(rightTable, foreignKeyExtractor, (leftValue, rightValue) -> join(leftValue, rightValue), Named.as("left-foreign-key-table-join")); + + //left join KStream-KTable + leftStream + .filter((key, value) -> key != null) + .leftJoin(kTable, (k, leftValue, rightValue) -> join(leftValue, rightValue)); + + //left join KStream-GlobalTable + KeyValueMapper<String;, String, String> keyValueMapper = (key, value) -> ...; + leftStream + .filter((key, value) -> keyValueMapper.apply(key,value) != null) + .leftJoin(globalTable, keyValueMapper, (leftValue, rightValue) -> join(leftValue, rightValue)); + + + +The `default.dsl.store` config was deprecated in favor of the new `dsl.store.suppliers.class` config to allow for custom state store implementations to be configured as the default. If you currently specify `default.dsl.store=ROCKS_DB` or `default.dsl.store=IN_MEMORY` replace those configurations with `dsl.store.suppliers.class=BuiltInDslStoreSuppliers.RocksDBDslStoreSuppliers.class` and `dsl.stores.suppliers.class=BuiltInDslStoreSuppliers.InMemoryDslStoreSuppliers.class` respectively + +A new configuration option `balance_subtopology` for `rack.aware.assignment.strategy` was introduced in 3.7 release. For more information, including how it can be enabled and further configured, see the [**Kafka Streams Developer Guide**](/37/streams/developer-guide/config-streams.html#rack-aware-assignment-strategy). + +## Streams API changes in 3.6.0 + +Rack aware task assignment was introduced in [KIP-925](https://cwiki.apache.org/confluence/display/KAFKA/KIP-925%3A+Rack+aware+task+assignment+in+Kafka+Streams). Rack aware task assignment can be enabled for `StickyTaskAssignor` or `HighAvailabilityTaskAssignor` to compute task assignments which can minimize cross rack traffic under certain conditions. For more information, including how it can be enabled and further configured, see the [**Kafka Streams Developer Guide**](/37/streams/developer-guide/config-streams.html#rack-aware-assignment-strategy). + +IQv2 supports a `RangeQuery` that allows to specify unbounded, bounded, or half-open key-ranges. Users have to use `withUpperBound(K)`, `withLowerBound(K)`, or `withNoBounds()` to specify half-open or unbounded ranges, but cannot use `withRange(K lower, K upper)` for the same. [KIP-941](https://cwiki.apache.org/confluence/display/KAFKA/KIP-941%3A+Range+queries+to+accept+null+lower+and+upper+bounds) closes this gap by allowing to pass in `null` as upper and lower bound (with semantics "no bound") to simplify the usage of the `RangeQuery` class. + +KStreams-to-KTable joins now have an option for adding a grace period. The grace period is enabled on the `Joined` object using with `withGracePeriod()` method. This change was introduced in [KIP-923](https://cwiki.apache.org/confluence/display/KAFKA/KIP-923%3A+Add+A+Grace+Period+to+Stream+Table+Join). To use the grace period option in the Stream-Table join the table must be [versioned](/37/streams/developer-guide/dsl-api.html#versioned-state-stores). For more information, including how it can be enabled and further configured, see the [**Kafka Streams Developer Guide**](/37/streams/developer-guide/config-streams.html#rack-aware-assignment-strategy). + +## Streams API changes in 3.5.0 + +A new state store type, versioned key-value stores, was introduced in [KIP-889](https://cwiki.apache.org/confluence/display/KAFKA/KIP-889%3A+Versioned+State+Stores) and [KIP-914](https://cwiki.apache.org/confluence/display/KAFKA/KIP-914%3A+DSL+Processor+Semantics+for+Versioned+Stores). Rather than storing a single record version (value and timestamp) per key, versioned state stores may store multiple record versions per key. This allows versioned state stores to support timestamped retrieval operations to return the latest record (per key) as of a specified timestamp. For more information, including how to upgrade from a non-versioned key-value store to a versioned store in an existing application, see the [Developer Guide](/37/streams/developer-guide/dsl-api.html#versioned-state-stores). Versioned key-value stores are opt-in only; existing applications will not be affected upon upgrading to 3.5 without explicit code changes. + +In addition to KIP-899, [KIP-914](https://cwiki.apache.org/confluence/display/KAFKA/KIP-914%3A+DSL+Processor+Semantics+for+Versioned+Stores) updates DSL processing semantics if a user opts-in to use the new versioned key-value stores. Using the new versioned key-value stores, DSL processing are able to handle out-of-order data better: For example, late record may be dropped and stream-table joins do a timestamped based lookup into the table. Table aggregations and primary/foreign-key table-table joins are also improved. Note: versioned key-value stores are not supported for global-KTable and don't work with `suppress()`. + +[KIP-904](https://cwiki.apache.org/confluence/display/KAFKA/KIP-904%3A+Kafka+Streams+-+Guarantee+subtractor+is+called+before+adder+if+key+has+not+changed) improves the implementation of KTable aggregations. In general, an input KTable update triggers a result refinent for two rows; however, prior to KIP-904, if both refinements happen to the same result row, two independent updates to the same row are applied, resulting in spurious itermediate results. KIP-904 allows us to detect this case, and to only apply a single update avoiding spurious intermediate results. + +Error handling is improved via [KIP-399](https://cwiki.apache.org/confluence/display/KAFKA/KIP-399%3A+Extend+ProductionExceptionHandler+to+cover+serialization+exceptions). The existing `ProductionExceptionHandler` now also covers serialization errors. + +We added a new Serde type `Boolean` in [KIP-907](https://cwiki.apache.org/confluence/display/KAFKA/KIP-907%3A+Add+Boolean+Serde+to+public+interface) + +[KIP-884](https://cwiki.apache.org/confluence/display/KAFKA/KIP-884%3A+Add+config+to+configure+KafkaClientSupplier+in+Kafka+Streams) adds a new config `default.client.supplier` that allows to use a custom `KafkaClientSupplier` without any code changes. + +## Streams API changes in 3.4.0 + +[KIP-770](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=186878390) deprecates config `cache.max.bytes.buffering` in favor of the newly introduced config `statestore.cache.max.bytes`. To improve monitoring, two new metrics `input-buffer-bytes-total` and `cache-size-bytes-total` were added at the DEBUG level. Note, that the KIP is only partially implemented in the 3.4.0 release, and config `input.buffer.max.bytes` is not available yet. + +[KIP-873](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211883356) enables you to multicast result records to multiple partition of downstream sink topics and adds functionality for choosing to drop result records without sending. The `Integer StreamPartitioner.partition()` method is deprecated and replaced by the newly added `Optiona≶Set>StreamPartitioner.partitions()` method, which enables returning a set of partitions to send the record to. + +[KIP-862](https://cwiki.apache.org/confluence/display/KAFKA/KIP-862%3A+Self-join+optimization+for+stream-stream+joins) adds a DSL optimization for stream-stream self-joins. The optimization is enabled via a new option `single.store.self.join` which can be set via existing config `topology.optimization`. If enabled, the DSL will use a different join processor implementation that uses a single RocksDB store instead of two, to avoid unnecessary data duplication for the self-join case. + +[KIP-865](https://cwiki.apache.org/confluence/display/KAFKA/KIP-865%3A+Support+--bootstrap-server+in+kafka-streams-application-reset) updates the Kafka Streams application reset tool’s server parameter name to conform to the other Kafka tooling by deprecating the `--bootstrap-servers` parameter and introducing a new `--bootstrap-server` parameter in its place. + +## Streams API changes in 3.3.0 + +Kafka Streams does not send a "leave group" request when an instance is closed. This behavior implies that a rebalance is delayed until `max.poll.interval.ms` passed. [KIP-812](https://cwiki.apache.org/confluence/display/KAFKA/KIP-812%3A+Introduce+another+form+of+the+%60KafkaStreams.close%28%29%60+API+that+forces+the+member+to+leave+the+consumer+group) introduces `KafkaStreams.close(CloseOptions)` overload, which allows forcing an instance to leave the group immediately. Note: Due to internal limitations, `CloseOptions` only works for static consumer groups at this point (cf. [KAFKA-16514](https://issues.apache.org/jira/browse/KAFKA-16514) for more details and a fix in some future release). + +[KIP-820](https://cwiki.apache.org/confluence/display/KAFKA/KIP-820%3A+Extend+KStream+process+with+new+Processor+API) adapts the PAPI type-safety improvement of KIP-478 into the DSL. The existing methods `KStream.transform`, `KStream.flatTransform`, `KStream.transformValues`, and `KStream.flatTransformValues` as well as all overloads of `void KStream.process` are deprecated in favor of the newly added methods + + * `KStream KStream.process(ProcessorSupplier, ...)` + * `KStream KStream.processValues(FixedKeyProcessorSupplier, ...)` + +Both new methods have multiple overloads and return a `KStream` instead of `void` as the deprecated `process()` methods did. In addition, `FixedKeyProcessor`, `FixedKeyRecord`, `FixedKeyProcessorContext`, and `ContextualFixedKeyProcessor` are introduced to guard against disallowed key modification inside `processValues()`. Furthermore, `ProcessingContext` is added for a better interface hierarchy. + +Emitting a windowed aggregation result only after a window is closed is currently supported via the `suppress()` operator. However, `suppress()` uses an in-memory implementation and does not support RocksDB. To close this gap, [KIP-825](https://cwiki.apache.org/confluence/display/KAFKA/KIP-825%3A+introduce+a+new+API+to+control+when+aggregated+results+are+produced) introduces "emit strategies", which are built into the aggregation operator directly to use the already existing RocksDB store. `TimeWindowedKStream.emitStrategy(EmitStrategy)` and `SessionWindowedKStream.emitStrategy(EmitStrategy)` allow picking between "emit on window update" (default) and "emit on window close" strategies. Additionally, a few new emit metrics are added, as well as a necessary new method, `SessionStore.findSessions(long, long)`. + +[KIP-834](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882832) allows pausing and resuming a Kafka Streams instance. Pausing implies that processing input records and executing punctuations will be skipped; Kafka Streams will continue to poll to maintain its group membership and may commit offsets. In addition to the new methods `KafkaStreams.pause()` and `KafkaStreams.resume()`, it is also supported to check if an instance is paused via the `KafkaStreams.isPaused()` method. + +To improve monitoring of Kafka Streams applications, [KIP-846](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211886093) adds four new metrics `bytes-consumed-total`, `records-consumed-total`, `bytes-produced-total`, and `records-produced-total` within a new **topic level** scope. The metrics are collected at INFO level for source and sink nodes, respectively. + +## Streams API changes in 3.2.0 + +RocksDB offers many metrics which are critical to monitor and tune its performance. Kafka Streams started to make RocksDB metrics accessible like any other Kafka metric via [KIP-471](https://cwiki.apache.org/confluence/display/KAFKA/KIP-471%3A+Expose+RocksDB+Metrics+in+Kafka+Streams) in 2.4.0 release. However, the KIP was only partially implemented, and is now completed with the 3.2.0 release. For a full list of available RocksDB metrics, please consult the [monitoring documentation](/37/#kafka_streams_client_monitoring). + +Kafka Streams ships with RocksDB and in-memory store implementations and users can pick which one to use. However, for the DSL, the choice is a per-operator one, making it cumbersome to switch from the default RocksDB store to in-memory store for all operators, especially for larger topologies. [KIP-591](https://cwiki.apache.org/confluence/display/KAFKA/KIP-591%3A+Add+Kafka+Streams+config+to+set+default+state+store) adds a new config `default.dsl.store` that enables setting the default store for all DSL operators globally. Note that it is required to pass `TopologyConfig` to the `StreamsBuilder` constructor to make use of this new config. + +For multi-AZ deployments, it is desired to assign StandbyTasks to a KafkaStreams instance running in a different AZ than the corresponding active StreamTask. [KIP-708](https://cwiki.apache.org/confluence/display/KAFKA/KIP-708%3A+Rack+aware+StandbyTask+assignment+for+Kafka+Streams) enables configuring Kafka Streams instances with a rack-aware StandbyTask assignment strategy, by using the new added configs `rack.aware.assignment.tags` and corresponding `client.tag.`. + +[KIP-791](https://cwiki.apache.org/confluence/display/KAFKA/KIP-791%3A+Add+Record+Metadata+to+State+Store+Context) adds a new method `Optional StateStoreContext.recordMetadata()` to expose record metadata. This helps for example to provide read-your-writes consistency guarantees in interactive queries. + +[Interactive Queries](/streams/developer-guide/interactive-queries.html) allow users to tap into the operational state of Kafka Streams processor nodes. The existing API is tightly coupled with the actual state store interfaces and thus the internal implementation of state store. To break up this tight coupling and allow for building more advanced IQ features, [KIP-796](https://cwiki.apache.org/confluence/display/KAFKA/KIP-796%3A+Interactive+Query+v2) introduces a completely new IQv2 API, via `StateQueryRequest` and `StateQueryResult` classes, as well as `Query` and `QueryResult` interfaces (plus additional helper classes). In addition, multiple built-in query types were added: `KeyQuery` for key lookups and `RangeQuery` (via [KIP-805](https://cwiki.apache.org/confluence/display/KAFKA/KIP-805%3A+Add+range+and+scan+query+over+kv-store+in+IQv2)) for key-range queries on key-value stores, as well as `WindowKeyQuery` and `WindowRangeQuery` (via [KIP-806](https://cwiki.apache.org/confluence/display/KAFKA/KIP-806%3A+Add+session+and+window+query+over+kv-store+in+IQv2)) for key and range lookup into windowed stores. + +The Kafka Streams DSL may insert so-called repartition topics for certain DSL operators to ensure correct partitioning of data. These topics are configured with infinite retention time, and Kafka Streams purges old data explicitly via "delete record" requests, when commiting input topic offsets. [KIP-811](https://cwiki.apache.org/confluence/display/KAFKA/KIP-811%3A+Add+config+repartition.purge.interval.ms+to+Kafka+Streams) adds a new config `repartition.purge.interval.ms` allowing you to configure the purge interval independently of the commit interval. + +## Streams API changes in 3.1.0 + +The semantics of left/outer stream-stream join got improved via [KIP-633](https://cwiki.apache.org/confluence/display/KAFKA/KIP-633%3A+Deprecate+24-hour+Default+Grace+Period+for+Windowed+Operations+in+Streams). Previously, left-/outer stream-stream join might have emitted so-call spurious left/outer results, due to an eager-emit strategy. The implementation was changed to emit left/outer join result records only after the join window is closed. The old API to specify the join window, i.e., `JoinWindows.of()` that enables the eager-emit strategy, was deprecated in favor of a `JoinWindows.ofTimeDifferenceAndGrace()` and `JoinWindows.ofTimeDifferencWithNoGrace()`. The new semantics are only enabled if you use the new join window builders. +Additionally, KIP-633 makes setting a grace period also mandatory for windowed aggregations, i.e., for `TimeWindows` (hopping/tumbling), `SessionWindows`, and `SlidingWindows`. The corresponding builder methods `.of(...)` were deprecated in favor of the new `.ofTimeDifferenceAndGrace()` and `.ofTimeDifferencWithNoGrace()` methods. + +[KIP-761](https://cwiki.apache.org/confluence/display/KAFKA/KIP-761%3A+Add+Total+Blocked+Time+Metric+to+Streams) adds new metrics that allow to track blocking times on the underlying consumer and producer clients. Check out the section on [Kafka Streams metrics](/#kafka_streams_monitoring) for more details. + +[Interactive Queries](/streams/developer-guide/interactive-queries.html) were improved via [KIP-763](https://cwiki.apache.org/confluence/display/KAFKA/KIP-763%3A+Range+queries+with+open+endpoints) [KIP-766](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=186876596). Range queries now accept `null` as lower/upper key-range bound to indicate an open-ended lower/upper bound. + +Foreign-key table-table joins now support custom partitioners via [KIP-775](https://cwiki.apache.org/confluence/display/KAFKA/KIP-775%3A+Custom+partitioners+in+foreign+key+joins). Previously, if an input table was partitioned by a non-default partitioner, joining records might fail. With KIP-775 you now can pass a custom `StreamPartitioner` into the join using the newly added `TableJoined` object. + +## Streams API changes in 3.0.0 + +We improved the semantics of [task idling (`max.task.idle.ms`)](/streams/developer-guide/config-streams.html#max-task-idle-ms). Now Streams provides stronger in-order join and merge processing semantics. Streams's new default pauses processing on tasks with multiple input partitions when one of the partitions has no data buffered locally but has a non-zero lag. In other words, Streams will wait to fetch records that are already available on the broker. This results in improved join semantics, since it allows Streams to interleave the two input partitions in timestamp order instead of just processing whichever partition happens to be buffered. There is an option to disable this new behavior, and there is also an option to make Streams wait even longer for new records to be _produced_ to the input partitions, which you can use to get stronger time semantics when you know some of your producers may be slow. See the [config reference](/streams/developer-guide/config-streams.html#max-task-idle-ms) for more information, and [KIP-695](https://cwiki.apache.org/confluence/x/JSXZCQ) for the larger context of this change. + +Interactive Queries may throw new exceptions for different errors: + + * `UnknownStateStoreException`: If the specified store name does not exist in the topology, an `UnknownStateStoreException` will be thrown instead of the former `InvalidStateStoreException`. + * `StreamsNotStartedException`: If Streams state is `CREATED`, a `StreamsNotStartedException` will be thrown. + * `InvalidStateStorePartitionException`: If the specified partition does not exist, a `InvalidStateStorePartitionException` will be thrown. + + + +See [KIP-216](https://cwiki.apache.org/confluence/display/KAFKA/KIP-216%3A+IQ+should+throw+different+exceptions+for+different+errors) for more information. + +We deprecated the StreamsConfig `processing.guarantee` configuration value `"exactly_once"` (for EOS version 1) in favor of the improved EOS version 2, formerly configured via `"exactly_once_beta`. To avoid confusion about the term "beta" in the config name and highlight the production-readiness of EOS version 2, we have also renamed "eos-beta" to "eos-v2" and deprecated the configuration value `"exactly_once_beta"`, replacing it with a new configuration value `"exactly_once_v2"` Users of exactly-once semantics should plan to migrate to the eos-v2 config and prepare for the removal of the deprecated configs in 4.0 or after at least a year from the release of 3.0, whichever comes last. Note that eos-v2 requires broker version 2.5 or higher, like eos-beta, so users should begin to upgrade their kafka cluster if necessary. See [KIP-732](https://cwiki.apache.org/confluence/x/zJONCg) for more details. + +We removed the default implementation of `RocksDBConfigSetter#close()`. + +We dropped the default 24 hours grace period for windowed operations such as Window or Session aggregates, or stream-stream joins. This period determines how long after a window ends any out-of-order records will still be processed. Records coming in after the grace period has elapsed are considered late and will be dropped. But in operators such as suppression, a large grace period has the drawback of incurring an equally large output latency. The current API made it all too easy to miss the grace period config completely, leading you to wonder why your application seems to produce no output -- it actually is, but not for 24 hours. + +To prevent accidentally or unknowingly falling back to the default 24hr grace period, we deprecated all of the existing static constructors for the `Windows` classes (such as `TimeWindows#of`). These are replaced by new static constructors of two flavors: `#ofSizeAndGrace` and `#ofSizeWithNoGrace` (these are for the `TimeWindows` class; analogous APIs exist for the `JoinWindows`, `SessionWindows`, and SlidingWindows classes). With these new APIs you are forced to set the grace period explicitly, or else consciously choose to opt out by selecting the `WithNoGrace` flavor which sets it to 0 for situations where you really don't care about the grace period, for example during testing or when playing around with Kafka Streams for the first time. Note that using the new APIs for the `JoinWindows` class will also enable a fix for spurious left/outer join results, as described in the following paragraph. For more details on the grace period and new static constructors, see [KIP-633](https://cwiki.apache.org/confluence/display/KAFKA/KIP-633%3A+Drop+24+hour+default+of+grace+period+in+Streams) + +Additionally, in older versions Kafka Streams emitted stream-stream left/outer join results eagerly. This behavior may lead to spurious left/outer join result records. In this release, we changed the behavior to avoid spurious results and left/outer join result are only emitted after the join window is closed, i.e., after the grace period elapsed. To maintain backward compatibility, the old API `JoinWindows#of(timeDifference)` preserves the old eager-emit behavior and only the new APIs `JoinWindows#ofTimeDifferenceAndGrace()` and `JoinsWindows#ofTimeDifferenceNoGrace` enable the new behavior. Check out [KAFKA-10847](https://issues.apache.org/jira/browse/KAFKA-10847) for more information. + +The public `topicGroupId` and `partition` fields on TaskId have been deprecated and replaced with getters. Please migrate to using the new `TaskId.subtopology()` (which replaces `topicGroupId`) and `TaskId.partition()` APIs instead. Also, the `TaskId#readFrom` and `TaskId#writeTo` methods have been deprecated and will be removed, as they were never intended for public use. We have also deprecated the `org.apache.kafka.streams.processor.TaskMetadata` class and introduced a new interface `org.apache.kafka.streams.TaskMetadata` to be used instead. This change was introduced to better reflect the fact that `TaskMetadata` was not meant to be instantiated outside of Kafka codebase. Please note that the new `TaskMetadata` offers APIs that better represent the task id as an actual `TaskId` object instead of a String. Please migrate to the new `org.apache.kafka.streams.TaskMetadata` which offers these better methods, for example, by using the new `ThreadMetadata#activeTasks` and `ThreadMetadata#standbyTasks`. `org.apache.kafka.streams.processor.ThreadMetadata` class is also now deprecated and the newly introduced interface `org.apache.kafka.streams.ThreadMetadata` is to be used instead. In this new `ThreadMetadata` interface, any reference to the deprecated `TaskMetadata` is replaced by the new interface. Finally, also `org.apache.kafka.streams.state.StreamsMetadata` has been deprecated. Please migrate to the new `org.apache.kafka.streams.StreamsMetadata`. We have deprecated several methods under `org.apache.kafka.streams.KafkaStreams` that returned the aforementioned deprecated classes: + + * Users of `KafkaStreams#allMetadata` are meant to migrate to the new `KafkaStreams#metadataForAllStreamsClients`. + * Users of `KafkaStreams#allMetadataForStore(String)` are meant to migrate to the new `KafkaStreams#streamsMetadataForStore(String)`. + * Users of `KafkaStreams#localThreadsMetadata` are meant to migrate to the new `KafkaStreams#metadataForLocalThreads`. + + + +See [KIP-740](https://cwiki.apache.org/confluence/x/vYTOCg) and [KIP-744](https://cwiki.apache.org/confluence/x/XIrOCg) for more details. + +We removed the following deprecated APIs: + + * `--zookeeper` flag of the application reset tool: deprecated in Kafka 1.0.0 ([KIP-198](https://cwiki.apache.org/confluence/display/KAFKA/KIP-198%3A+Remove+ZK+dependency+from+Streams+Reset+Tool)). + * `--execute` flag of the application reset tool: deprecated in Kafka 1.1.0 ([KIP-171](https://cwiki.apache.org/confluence/display/KAFKA/KIP-171+-+Extend+Consumer+Group+Reset+Offset+for+Stream+Application)). + * `StreamsBuilder#addGlobalStore` (one overload): deprecated in Kafka 1.1.0 ([KIP-233](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=74689212)). + * `ProcessorContext#forward` (some overloads): deprecated in Kafka 2.0.0 ([KIP-251](https://cwiki.apache.org/confluence/display/KAFKA/KIP-251%3A+Allow+timestamp+manipulation+in+Processor+API)). + * `WindowBytesStoreSupplier#segments`: deprecated in Kafka 2.1.0 ([KIP-319](https://cwiki.apache.org/confluence/display/KAFKA/KIP-319%3A+Replace+segments+with+segmentInterval+in+WindowBytesStoreSupplier)). + * `segments, until, maintainMs` on `TimeWindows`, `JoinWindows`, and `SessionWindows`: deprecated in Kafka 2.1.0 ([KIP-328](https://cwiki.apache.org/confluence/display/KAFKA/KIP-328%3A+Ability+to+suppress+updates+for+KTables)). + * Overloaded `JoinWindows#of, before, after`, `SessionWindows#with`, `TimeWindows#of, advanceBy`, `UnlimitedWindows#startOn` and `KafkaStreams#close` with `long` typed parameters: deprecated in Kafka 2.1.0 ([KIP-358](https://cwiki.apache.org/confluence/display/KAFKA/KIP-358%3A+Migrate+Streams+API+to+Duration+instead+of+long+ms+times)). + * Overloaded `KStream#groupBy, groupByKey` and `KTable#groupBy` with `Serialized` parameter: deprecated in Kafka 2.1.0 ([KIP-372](https://cwiki.apache.org/confluence/display/KAFKA/KIP-372%3A+Naming+Repartition+Topics+for+Joins+and+Grouping)). + * `Joined#named, name`: deprecated in Kafka 2.3.0 ([KIP-307](https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL)). + * `TopologyTestDriver#pipeInput, readOutput`, `OutputVerifier` and `ConsumerRecordFactory` classes ([KIP-470](https://cwiki.apache.org/confluence/display/KAFKA/KIP-470%3A+TopologyTestDriver+test+input+and+output+usability+improvements)). + * `KafkaClientSupplier#getAdminClient`: deprecated in Kafka 2.4.0 ([KIP-476](https://cwiki.apache.org/confluence/display/KAFKA/KIP-476%3A+Add+Java+AdminClient+Interface)). + * Overloaded `KStream#join, leftJoin, outerJoin` with `KStream` and `Joined` parameters: deprecated in Kafka 2.4.0 ([KIP-479](https://cwiki.apache.org/confluence/display/KAFKA/KIP-479%3A+Add+StreamJoined+config+object+to+Join)). + * `WindowStore#put(K key, V value)`: deprecated in Kafka 2.4.0 ([KIP-474](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=115526545)). + * `UsePreviousTimeOnInvalidTimestamp`: deprecated in Kafka 2.5.0 as renamed to `UsePartitionTimeOnInvalidTimestamp` ([KIP-530](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=130028807)). + * Overloaded `KafkaStreams#metadataForKey`: deprecated in Kafka 2.5.0 ([KIP-535](https://cwiki.apache.org/confluence/display/KAFKA/KIP-535%3A+Allow+state+stores+to+serve+stale+reads+during+rebalance)). + * Overloaded `KafkaStreams#store`: deprecated in Kafka 2.5.0 ([KIP-562](https://cwiki.apache.org/confluence/display/KAFKA/KIP-562%3A+Allow+fetching+a+key+from+a+single+partition+rather+than+iterating+over+all+the+stores+on+an+instance)). + + + +The following dependencies were removed from Kafka Streams: + + * Connect-json: As of Kafka Streams no longer has a compile time dependency on "connect:json" module ([KAFKA-5146](https://issues.apache.org/jira/browse/KAFKA-5146)). Projects that were relying on this transitive dependency will have to explicitly declare it. + + + +The default value for configuration parameter `replication.factor` was changed to `-1` (meaning: use broker default replication factor). The `replication.factor` value of `-1` requires broker version 2.4 or newer. + +The new serde type was introduced `ListSerde`: + + * Added class `ListSerde` to (de)serialize `List`-based objects + * Introduced `ListSerializer` and `ListDeserializer` to power the new functionality + + + +## Streams API changes in 2.8.0 + +We extended `StreamJoined` to include the options `withLoggingEnabled()` and `withLoggingDisabled()` in [KIP-689](https://cwiki.apache.org/confluence/display/KAFKA/KIP-689%3A+Extend+%60StreamJoined%60+to+allow+more+store+configs). + +We added two new methods to `KafkaStreams`, namely `KafkaStreams#addStreamThread()` and `KafkaStreams#removeStreamThread()` in [KIP-663](https://cwiki.apache.org/confluence/x/FDd4CQ). These methods have enabled adding and removing StreamThreads to a running KafkaStreams client. + +We deprecated `KafkaStreams#setUncaughtExceptionHandler(final Thread.UncaughtExceptionHandler uncaughtExceptionHandler)` in favor of `KafkaStreams#setUncaughtExceptionHandler(final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler)` in [KIP-671](https://cwiki.apache.org/confluence/x/lkN4CQ). The default handler will close the Kafka Streams client and the client will transit to state ERROR. If you implement a custom handler, the new interface allows you to return a `StreamThreadExceptionResponse`, which will determine how the application will respond to a stream thread failure. + +Changes in [KIP-663](https://cwiki.apache.org/confluence/x/FDd4CQ) necessitated the KafkaStreams client state machine to update, which was done in [KIP-696](https://cwiki.apache.org/confluence/x/lCvZCQ). The ERROR state is now terminal with PENDING_ERROR being a transitional state where the resources are closing. The ERROR state indicates that there is something wrong and the Kafka Streams client should not be blindly restarted without classifying the error that caused the thread to fail. If the error is of a type that you would like to retry, you should have the `StreamsUncaughtExceptionHandler` return `REPLACE_THREAD`. When all stream threads are dead there is no automatic transition to ERROR as a new stream thread can be added. + +The `TimeWindowedDeserializer` constructor `TimeWindowedDeserializer(final Deserializer inner)` was deprecated to encourage users to properly set their window size through `TimeWindowedDeserializer(final Deserializer inner, Long windowSize)`. An additional streams config, `window.size.ms`, was added for users that cannot set the window size through the constructor, such as when using the console consumer. [KIP-659](https://cwiki.apache.org/confluence/display/KAFKA/KIP-659%3A+Improve+TimeWindowedDeserializer+and+TimeWindowedSerde+to+handle+window+size) has more details. + +To simplify testing, two new constructors that don't require a `Properties` parameter have been added to the `TopologyTestDriver` class. If `Properties` are passed into the constructor, it is no longer required to set mandatory configuration parameters (cf. [KIP-680](https://cwiki.apache.org/confluence/display/KAFKA/KIP-680%3A+TopologyTestDriver+should+not+require+a+Properties+argument)). + +We added the `prefixScan()` method to interface `ReadOnlyKeyValueStore`. The new `prefixScan()` allows fetching all values whose keys start with a given prefix. See [KIP-614](https://cwiki.apache.org/confluence/display/KAFKA/KIP-614%3A+Add+Prefix+Scan+support+for+State+Stores) for more details. + +Kafka Streams is now handling `TimeoutException` thrown by the consumer, producer, and admin client. If a timeout occurs on a task, Kafka Streams moves to the next task and retries to make progress on the failed task in the next iteration. To bound how long Kafka Streams retries a task, you can set `task.timeout.ms` (default is 5 minutes). If a task does not make progress within the specified task timeout, which is tracked on a per-task basis, Kafka Streams throws a `TimeoutException` (cf. [KIP-572](https://cwiki.apache.org/confluence/display/KAFKA/KIP-572%3A+Improve+timeouts+and+retries+in+Kafka+Streams)). + +We changed the default value of `default.key.serde` and `default.value.serde` to be `null` instead of `ByteArraySerde`. Users will now see a `ConfigException` if their serdes are not correctly configured through those configs or passed in explicitly. See [KIP-741](https://cwiki.apache.org/confluence/display/KAFKA/KIP-741%3A+Change+default+serde+to+be+null) for more details. + +## Streams API changes in 2.7.0 + +In `KeyQueryMetadata` we deprecated `getActiveHost()`, `getStandbyHosts()` as well as `getPartition()` and replaced them with `activeHost()`, `standbyHosts()` and `partition()` respectively. `KeyQueryMetadata` was introduced in Kafka Streams 2.5 release with getter methods having prefix `get`. The intend of this change is to bring the method names to Kafka custom to not use the `get` prefix for getter methods. The old methods are deprecated and is not effected. (Cf. [KIP-648](https://cwiki.apache.org/confluence/display/KAFKA/KIP-648%3A+Renaming+getter+method+for+Interactive+Queries).) + +The `StreamsConfig` variable for configuration parameter `"topology.optimization"` is renamed from `TOPOLOGY_OPTIMIZATION` to `TOPOLOGY_OPTIMIZATION_CONFIG`. The old variable is deprecated. Note, that the parameter name itself is not affected. (Cf. [KIP-629](https://cwiki.apache.org/confluence/display/KAFKA/KIP-626%3A+Rename+StreamsConfig+config+variable+name).) + +The configuration parameter `retries` is deprecated in favor of the new parameter `task.timeout.ms`. Kafka Streams' runtime ignores `retries` if set, however, it would still forward the parameter to its internal clients. + +We added `SlidingWindows` as an option for `windowedBy()` windowed aggregations as described in [KIP-450](https://cwiki.apache.org/confluence/display/KAFKA/KIP-450%3A+Sliding+Window+Aggregations+in+the+DSL). Sliding windows are fixed-time and data-aligned windows that allow for flexible and efficient windowed aggregations. + +The end-to-end latency metrics introduced in 2.6 have been expanded to include store-level metrics. The new store-level metrics are recorded at the TRACE level, a new metrics recording level. Enabling TRACE level metrics will automatically turn on all higher levels, ie INFO and DEBUG. See [KIP-613](https://cwiki.apache.org/confluence/x/gBkRCQ) for more information. + +## Streams API changes in 2.6.0 + +We added a new processing mode, EOS version 2, that improves application scalability using exactly-once guarantees (via [KIP-447](https://cwiki.apache.org/confluence/display/KAFKA/KIP-447%3A+Producer+scalability+for+exactly+once+semantics)). You can enable this new feature by setting the configuration parameter `processing.guarantee` to the new value `"exactly_once_beta"`. Note that you need brokers with version 2.5 or newer to use this feature. + +For more highly available stateful applications, we've modified the task assignment algorithm to delay the movement of stateful active tasks to instances that aren't yet caught up with that task's state. Instead, to migrate a task from one instance to another (eg when scaling out), Streams will assign a warmup replica to the target instance so it can begin restoring the state while the active task stays available on an instance that already had the task. The instances warming up tasks will communicate their progress to the group so that, once ready, Streams can move active tasks to their new owners in the background. Check out [KIP-441](https://cwiki.apache.org/confluence/x/0i4lBg) for full details, including several new configs for control over this new feature. + +New end-to-end latency metrics have been added. These task-level metrics will be logged at the INFO level and report the min and max end-to-end latency of a record at the beginning/source node(s) and end/terminal node(s) of a task. See [KIP-613](https://cwiki.apache.org/confluence/x/gBkRCQ) for more information. + +As of 2.6.0 Kafka Streams deprecates `KStream.through()` in favor of the new `KStream.repartition()` operator (as per [KIP-221](https://cwiki.apache.org/confluence/display/KAFKA/KIP-221%3A+Enhance+DSL+with+Connecting+Topic+Creation+and+Repartition+Hint)). `KStream.repartition()` is similar to `KStream.through()`, however Kafka Streams will manage the topic for you. If you need to write into and read back from a topic that you mange, you can fall back to use `KStream.to()` in combination with `StreamsBuilder#stream()`. Please refer to the [developer guide](/37/streams/developer-guide/dsl-api.html) for more details about `KStream.repartition()`. + +The usability of `StateStore`s within the Processor API is improved: `ProcessorSupplier` and `TransformerSupplier` now extend `ConnectedStoreProvider` as per [KIP-401](https://cwiki.apache.org/confluence/x/XI3QBQ), enabling a user to provide `StateStore`s with alongside Processor/Transformer logic so that they are automatically added and connected to the processor. + +We added a `--force` option in StreamsResetter to force remove left-over members on broker side when long session time out was configured as per [KIP-571](https://cwiki.apache.org/confluence/display/KAFKA/KIP-571%3A+Add+option+to+force+remove+members+in+StreamsResetter). + +We added `Suppressed.withLoggingDisabled()` and `Suppressed.withLoggingEnabled(config)` methods to allow disabling or configuring of the changelog topic and allows for configuration of the changelog topic as per [KIP-446](https://cwiki.apache.org/confluence/display/KAFKA/KIP-446%3A+Add+changelog+topic+configuration+to+KTable+suppress). + +## Streams API changes in 2.5.0 + +We add a new `cogroup()` operator (via [KIP-150](https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup)) that allows to aggregate multiple streams in a single operation. Cogrouped streams can also be windowed before they are aggregated. Please refer to the [developer guide](/37/streams/developer-guide/dsl-api.html) for more details. + +We added a new `KStream.toTable()` API to translate an input event stream into a changelog stream as per [KIP-523](https://cwiki.apache.org/confluence/display/KAFKA/KIP-523%3A+Add+KStream%23toTable+to+the+Streams+DSL). + +We added a new Serde type `Void` in [KIP-527](https://cwiki.apache.org/confluence/display/KAFKA/KIP-527%3A+Add+VoidSerde+to+Serdes) to represent null keys or null values from input topic. + +Deprecated `UsePreviousTimeOnInvalidTimestamp` and replaced it with `UsePartitionTimeOnInvalidTimeStamp` as per [KIP-530](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=130028807). + +Deprecated `KafkaStreams.store(String, QueryableStoreType)` and replaced it with `KafkaStreams.store(StoreQueryParameters)` to allow querying for a store with variety of parameters, including querying a specific task and stale stores, as per [KIP-562](https://cwiki.apache.org/confluence/display/KAFKA/KIP-562%3A+Allow+fetching+a+key+from+a+single+partition+rather+than+iterating+over+all+the+stores+on+an+instance) and [KIP-535](https://cwiki.apache.org/confluence/display/KAFKA/KIP-535%3A+Allow+state+stores+to+serve+stale+reads+during+rebalance) respectively. + +## Streams API changes in 2.4.0 + +As of 2.4.0 Kafka Streams offers a KTable-KTable foreign-key join (as per [KIP-213](https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable)). This joiner allows for records to be joined between two KTables with different keys. Both [INNER and LEFT foreign-key joins](/37/streams/developer-guide/dsl-api.html#ktable-ktable-fk-join) are supported. + +In the 2.4 release, you now can name all operators in a Kafka Streams DSL topology via [KIP-307](https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL). Giving your operators meaningful names makes it easier to understand the topology description (`Topology#describe()#toString()`) and understand the full context of what your Kafka Streams application is doing. +There are new overloads on most `KStream` and `KTable` methods that accept a `Named` object. Typically you'll provide a name for the DSL operation by using `Named.as("my operator name")`. Naming of repartition topics for aggregation operations will still use `Grouped` and join operations will use either `Joined` or the new `StreamJoined` object. + +Before the 2.4.0 version of Kafka Streams, users of the DSL could not name the state stores involved in a stream-stream join. If users changed their topology and added a operator before the join, the internal names of the state stores would shift, requiring an application reset when redeploying. In the 2.4.0 release, Kafka Streams adds the `StreamJoined` class, which gives users the ability to name the join processor, repartition topic(s) (if a repartition is required), and the state stores involved in the join. Also, by naming the state stores, the changelog topics backing the state stores are named as well. It's important to note that naming the stores **will not** make them queryable via Interactive Queries. +Another feature delivered by `StreamJoined` is that you can now configure the type of state store used in the join. You can elect to use in-memory stores or custom state stores for a stream-stream join. Note that the provided stores will not be available for querying via Interactive Queries. With the addition of `StreamJoined`, stream-stream join operations using `Joined` have been deprecated. Please switch over to stream-stream join methods using the new overloaded methods. You can get more details from [KIP-479](https://cwiki.apache.org/confluence/display/KAFKA/KIP-479%3A+Add+StreamJoined+config+object+to+Join). + +With the introduction of incremental cooperative rebalancing, Streams no longer requires all tasks be revoked at the beginning of a rebalance. Instead, at the completion of the rebalance only those tasks which are to be migrated to another consumer for overall load balance will need to be closed and revoked. This changes the semantics of the `StateListener` a bit, as it will not necessarily transition to `REBALANCING` at the beginning of a rebalance anymore. Note that this means IQ will now be available at all times except during state restoration, including while a rebalance is in progress. If restoration is occurring when a rebalance begins, we will continue to actively restore the state stores and/or process standby tasks during a cooperative rebalance. Note that with this new rebalancing protocol, you may sometimes see a rebalance be followed by a second short rebalance that ensures all tasks are safely distributed. For details on please see [KIP-429](https://cwiki.apache.org/confluence/display/KAFKA/KIP-429%3A+Kafka+Consumer+Incremental+Rebalance+Protocol). + +The 2.4.0 release contains newly added and reworked metrics. [KIP-444](https://cwiki.apache.org/confluence/display/KAFKA/KIP-444%3A+Augment+metrics+for+Kafka+Streams) adds new _client level_ (i.e., `KafkaStreams` instance level) metrics to the existing thread-level, task-level, and processor-/state-store-level metrics. For a full list of available client level metrics, see the [KafkaStreams monitoring](/37/#kafka_streams_client_monitoring) section in the operations guide. +Furthermore, RocksDB metrics are exposed via [KIP-471](https://cwiki.apache.org/confluence/display/KAFKA/KIP-471%3A+Expose+RocksDB+Metrics+in+Kafka+Streams). For a full list of available RocksDB metrics, see the [RocksDB monitoring](/37/#kafka_streams_rocksdb_monitoring) section in the operations guide. + +Kafka Streams `test-utils` got improved via [KIP-470](https://cwiki.apache.org/confluence/display/KAFKA/KIP-470%3A+TopologyTestDriver+test+input+and+output+usability+improvements) to simplify the process of using `TopologyTestDriver` to test your application code. We deprecated `ConsumerRecordFactory`, `TopologyTestDriver#pipeInput()`, `OutputVerifier`, as well as `TopologyTestDriver#readOutput()` and replace them with `TestInputTopic` and `TestOutputTopic`, respectively. We also introduced a new class `TestRecord` that simplifies assertion code. For full details see the [Testing section](/37/streams/developer-guide/testing.html) in the developer guide. + +In 2.4.0, we deprecated `WindowStore#put(K key, V value)` that should never be used. Instead the existing `WindowStore#put(K key, V value, long windowStartTimestamp)` should be used ([KIP-474](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=115526545)). + +Furthermore, the `PartitionGrouper` interface and its corresponding configuration parameter `partition.grouper` were deprecated ([KIP-528](https://cwiki.apache.org/confluence/display/KAFKA/KIP-528%3A+Deprecate+PartitionGrouper+configuration+and+interface)) and will be removed in the next major release ([KAFKA-7785](https://issues.apache.org/jira/browse/KAFKA-7785). Hence, this feature won't be supported in the future any longer and you need to updated your code accordingly. If you use a custom `PartitionGrouper` and stop to use it, the created tasks might change. Hence, you will need to reset your application to upgrade it. + +## Streams API changes in 2.3.0 + +Version 2.3.0 adds the Suppress operator to the `kafka-streams-scala` Ktable API. + +As of 2.3.0 Streams now offers an in-memory version of the window ([KIP-428](https://cwiki.apache.org/confluence/display/KAFKA/KIP-428%3A+Add+in-memory+window+store)) and the session ([KIP-445](https://cwiki.apache.org/confluence/display/KAFKA/KIP-445%3A+In-memory+Session+Store)) store, in addition to the persistent ones based on RocksDB. The new public interfaces `inMemoryWindowStore()` and `inMemorySessionStore()` are added to `Stores` and provide the built-in in-memory window or session store. + +As of 2.3.0 we've updated how to turn on optimizations. Now to enable optimizations, you need to do two things. First add this line to your properties `properties.setProperty(StreamsConfig.TOPOLOGY_OPTIMIZATION, StreamsConfig.OPTIMIZE);`, as you have done before. Second, when constructing your `KafkaStreams` instance, you'll need to pass your configuration properties when building your topology by using the overloaded `StreamsBuilder.build(Properties)` method. For example `KafkaStreams myStream = new KafkaStreams(streamsBuilder.build(properties), properties)`. + +In 2.3.0 we have added default implementation to `close()` and `configure()` for `Serializer`, `Deserializer` and `Serde` so that they can be implemented by lambda expression. For more details please read [KIP-331](https://cwiki.apache.org/confluence/display/KAFKA/KIP-331+Add+default+implementation+to+close%28%29+and+configure%28%29+for+Serializer%2C+Deserializer+and+Serde). + +To improve operator semantics, new store types are added that allow storing an additional timestamp per key-value pair or window. Some DSL operators (for example KTables) are using those new stores. Hence, you can now retrieve the last update timestamp via Interactive Queries if you specify `TimestampedKeyValueStoreType` or `TimestampedWindowStoreType` as your `QueryableStoreType`. While this change is mainly transparent, there are some corner cases that may require code changes: **Caution: If you receive an untyped store and use a cast, you might need to update your code to cast to the correct type. Otherwise, you might get an exception similar to`java.lang.ClassCastException: class org.apache.kafka.streams.state.ValueAndTimestamp cannot be cast to class YOUR-VALUE-TYPE` upon getting a value from the store.** Additionally, `TopologyTestDriver#getStateStore()` only returns non-built-in stores and throws an exception if a built-in store is accessed. For more details please read [KIP-258](https://cwiki.apache.org/confluence/display/KAFKA/KIP-258%3A+Allow+to+Store+Record+Timestamps+in+RocksDB). + +To improve type safety, a new operator `KStream#flatTransformValues` is added. For more details please read [KIP-313](https://cwiki.apache.org/confluence/display/KAFKA/KIP-313%3A+Add+KStream.flatTransform+and+KStream.flatTransformValues). + +Kafka Streams used to set the configuration parameter `max.poll.interval.ms` to `Integer.MAX_VALUE`. This default value is removed and Kafka Streams uses the consumer default value now. For more details please read [KIP-442](https://cwiki.apache.org/confluence/display/KAFKA/KIP-442%3A+Return+to+default+max+poll+interval+in+Streams). + +Default configuration for repartition topic was changed: The segment size for index files (`segment.index.bytes`) is no longer 50MB, but uses the cluster default. Similarly, the configuration `segment.ms` in no longer 10 minutes, but uses the cluster default configuration. Lastly, the retention period (`retention.ms`) is changed from `Long.MAX_VALUE` to `-1` (infinite). For more details please read [KIP-443](https://cwiki.apache.org/confluence/display/KAFKA/KIP-443%3A+Return+to+default+segment.ms+and+segment.index.bytes+in+Streams+repartition+topics). + +To avoid memory leaks, `RocksDBConfigSetter` has a new `close()` method that is called on shutdown. Users should implement this method to release any memory used by RocksDB config objects, by closing those objects. For more details please read [KIP-453](https://cwiki.apache.org/confluence/display/KAFKA/KIP-453%3A+Add+close%28%29+method+to+RocksDBConfigSetter). + +RocksDB dependency was updated to version `5.18.3`. The new version allows to specify more RocksDB configurations, including `WriteBufferManager` which helps to limit RocksDB off-heap memory usage. For more details please read [KAFKA-8215](https://issues.apache.org/jira/browse/KAFKA-8215). + +## Streams API changes in 2.2.0 + +We've simplified the `KafkaStreams#state` transition diagram during the starting up phase a bit in 2.2.0: in older versions the state will transit from `CREATED` to `RUNNING`, and then to `REBALANCING` to get the first stream task assignment, and then back to `RUNNING`; starting in 2.2.0 it will transit from `CREATED` directly to `REBALANCING` and then to `RUNNING`. If you have registered a `StateListener` that captures state transition events, you may need to adjust your listener implementation accordingly for this simplification (in practice, your listener logic should be very unlikely to be affected at all). + +In `WindowedSerdes`, we've added a new static constructor to return a `TimeWindowSerde` with configurable window size. This is to help users to construct time window serdes to read directly from a time-windowed store's changelog. More details can be found in [KIP-393](https://cwiki.apache.org/confluence/display/KAFKA/KIP-393%3A+Time+windowed+serde+to+properly+deserialize+changelog+input+topic). + +In 2.2.0 we have extended a few public interfaces including `KafkaStreams` to extend `AutoCloseable` so that they can be used in a try-with-resource statement. For a full list of public interfaces that get impacted please read [KIP-376](https://cwiki.apache.org/confluence/display/KAFKA/KIP-376%3A+Implement+AutoClosable+on+appropriate+classes+that+want+to+be+used+in+a+try-with-resource+statement). + +## Streams API changes in 2.1.0 + +We updated `TopologyDescription` API to allow for better runtime checking. Users are encouraged to use `#topicSet()` and `#topicPattern()` accordingly on `TopologyDescription.Source` nodes, instead of using `#topics()`, which has since been deprecated. Similarly, use `#topic()` and `#topicNameExtractor()` to get descriptions of `TopologyDescription.Sink` nodes. For more details, see [KIP-321](https://cwiki.apache.org/confluence/display/KAFKA/KIP-321%3A+Update+TopologyDescription+to+better+represent+Source+and+Sink+Nodes). + +We've added a new class `Grouped` and deprecated `Serialized`. The intent of adding `Grouped` is the ability to name repartition topics created when performing aggregation operations. Users can name the potential repartition topic using the `Grouped#as()` method which takes a `String` and is used as part of the repartition topic name. The resulting repartition topic name will still follow the pattern of `${application-id}->name<-repartition`. The `Grouped` class is now favored over `Serialized` in `KStream#groupByKey()`, `KStream#groupBy()`, and `KTable#groupBy()`. Note that Kafka Streams does not automatically create repartition topics for aggregation operations. Additionally, we've updated the `Joined` class with a new method `Joined#withName` enabling users to name any repartition topics required for performing Stream/Stream or Stream/Table join. For more details repartition topic naming, see [KIP-372](https://cwiki.apache.org/confluence/display/KAFKA/KIP-372%3A+Naming+Repartition+Topics+for+Joins+and+Grouping). As a result we've updated the Kafka Streams Scala API and removed the `Serialized` class in favor of adding `Grouped`. If you just rely on the implicit `Serialized`, you just need to recompile; if you pass in `Serialized` explicitly, sorry you'll have to make code changes. + +We've added a new config named `max.task.idle.ms` to allow users specify how to handle out-of-order data within a task that may be processing multiple topic-partitions (see [Out-of-Order Handling](/37/streams/core-concepts.html#streams_out_of_ordering) section for more details). The default value is set to `0`, to favor minimized latency over synchronization between multiple input streams from topic-partitions. If users would like to wait for longer time when some of the topic-partitions do not have data available to process and hence cannot determine its corresponding stream time, they can override this config to a larger value. + +We've added the missing `SessionBytesStoreSupplier#retentionPeriod()` to be consistent with the `WindowBytesStoreSupplier` which allows users to get the specified retention period for session-windowed stores. We've also added the missing `StoreBuilder#withCachingDisabled()` to allow users to turn off caching for their customized stores. + +We added a new serde for UUIDs (`Serdes.UUIDSerde`) that you can use via `Serdes.UUID()` (cf. [KIP-206](https://cwiki.apache.org/confluence/display/KAFKA/KIP-206%3A+Add+support+for+UUID+serialization+and+deserialization)). + +We updated a list of methods that take `long` arguments as either timestamp (fix point) or duration (time period) and replaced them with `Instant` and `Duration` parameters for improved semantics. Some old methods base on `long` are deprecated and users are encouraged to update their code. +In particular, aggregation windows (hopping/tumbling/unlimited time windows and session windows) as well as join windows now take `Duration` arguments to specify window size, hop, and gap parameters. Also, window sizes and retention times are now specified as `Duration` type in `Stores` class. The `Window` class has new methods `#startTime()` and `#endTime()` that return window start/end timestamp as `Instant`. For interactive queries, there are new `#fetch(...)` overloads taking `Instant` arguments. Additionally, punctuations are now registered via `ProcessorContext#schedule(Duration interval, ...)`. For more details, see [KIP-358](https://cwiki.apache.org/confluence/display/KAFKA/KIP-358%3A+Migrate+Streams+API+to+Duration+instead+of+long+ms+times). + +We deprecated `KafkaStreams#close(...)` and replaced it with `KafkaStreams#close(Duration)` that accepts a single timeout argument Note: the new `#close(Duration)` method has improved (but slightly different) semantics. For more details, see [KIP-358](https://cwiki.apache.org/confluence/display/KAFKA/KIP-358%3A+Migrate+Streams+API+to+Duration+instead+of+long+ms+times). + +The newly exposed `AdminClient` metrics are now available when calling the `KafkaStream#metrics()` method. For more details on exposing `AdminClients` metrics see [KIP-324](https://cwiki.apache.org/confluence/display/KAFKA/KIP-324%3A+Add+method+to+get+metrics%28%29+in+AdminClient) + +We deprecated the notion of segments in window stores as those are intended to be an implementation details. Thus, method `Windows#segments()` and variable `Windows#segments` were deprecated. If you implement custom windows, you should update your code accordingly. Similarly, `WindowBytesStoreSupplier#segments()` was deprecated and replaced with `WindowBytesStoreSupplier#segmentInterval()`. If you implement custom window store, you need to update your code accordingly. Finally, `Stores#persistentWindowStore(...)` were deprecated and replaced with a new overload that does not allow to specify the number of segments any longer. For more details, see [KIP-319](https://cwiki.apache.org/confluence/display/KAFKA/KIP-319%3A+Replace+segments+with+segmentInterval+in+WindowBytesStoreSupplier) (note: [KIP-328](https://cwiki.apache.org/confluence/display/KAFKA/KIP-328%3A+Ability+to+suppress+updates+for+KTables) and [KIP-358](https://cwiki.apache.org/confluence/display/KAFKA/KIP-358%3A+Migrate+Streams+API+to+Duration+instead+of+long+ms+times) 'overlap' with KIP-319). + +We've added an overloaded `StreamsBuilder#build` method that accepts an instance of `java.util.Properties` with the intent of using the `StreamsConfig#TOPOLOGY_OPTIMIZATION` config added in Kafka Streams 2.0. Before 2.1, when building a topology with the DSL, Kafka Streams writes the physical plan as the user makes calls on the DSL. Now by providing a `java.util.Properties` instance when executing a `StreamsBuilder#build` call, Kafka Streams can optimize the physical plan of the topology, provided the `StreamsConfig#TOPOLOGY_OPTIMIZATION` config is set to `StreamsConfig#OPTIMIZE`. By setting `StreamsConfig#OPTIMIZE` in addition to the `KTable` optimization of reusing the source topic as the changelog topic, the topology may be optimized to merge redundant repartition topics into one repartition topic. The original no parameter version of `StreamsBuilder#build` is still available for those who wish to not optimize their topology. Note that enabling optimization of the topology may require you to do an application reset when redeploying the application. For more details, see [KIP-312](https://cwiki.apache.org/confluence/display/KAFKA/KIP-312%3A+Add+Overloaded+StreamsBuilder+Build+Method+to+Accept+java.util.Properties) + +We are introducing static membership towards Kafka Streams user. This feature reduces unnecessary rebalances during normal application upgrades or rolling bounces. For more details on how to use it, checkout [static membership design](/37/#static_membership). Note, Kafka Streams uses the same `ConsumerConfig#GROUP_INSTANCE_ID_CONFIG`, and you only need to make sure it is uniquely defined across different stream instances in one application. + +## Streams API changes in 2.0.0 + +In 2.0.0 we have added a few new APIs on the `ReadOnlyWindowStore` interface (for details please read Streams API changes below). If you have customized window store implementations that extends the `ReadOnlyWindowStore` interface you need to make code changes. + +In addition, if you using Java 8 method references in your Kafka Streams code you might need to update your code to resolve method ambiguities. Hot-swapping the jar-file only might not work for this case. See below a complete list of 2.0.0 API and semantic changes that allow you to advance your application and/or simplify your code base. + +We moved `Consumed` interface from `org.apache.kafka.streams` to `org.apache.kafka.streams.kstream` as it was mistakenly placed in the previous release. If your code has already used it there is a simple one-liner change needed in your import statement. + +We have also removed some public APIs that are deprecated prior to 1.0.x in 2.0.0. See below for a detailed list of removed APIs. + +We have removed the `skippedDueToDeserializationError-rate` and `skippedDueToDeserializationError-total` metrics. Deserialization errors, and all other causes of record skipping, are now accounted for in the pre-existing metrics `skipped-records-rate` and `skipped-records-total`. When a record is skipped, the event is now logged at WARN level. If these warnings become burdensome, we recommend explicitly filtering out unprocessable records instead of depending on record skipping semantics. For more details, see [KIP-274](https://cwiki.apache.org/confluence/display/KAFKA/KIP-274%3A+Kafka+Streams+Skipped+Records+Metrics). As of right now, the potential causes of skipped records are: + + * `null` keys in table sources + * `null` keys in table-table inner/left/outer/right joins + * `null` keys or values in stream-table joins + * `null` keys or values in stream-stream joins + * `null` keys or values in aggregations on grouped streams + * `null` keys or values in reductions on grouped streams + * `null` keys in aggregations on windowed streams + * `null` keys in reductions on windowed streams + * `null` keys in aggregations on session-windowed streams + * Errors producing results, when the configured `default.production.exception.handler` decides to `CONTINUE` (the default is to `FAIL` and throw an exception). + * Errors deserializing records, when the configured `default.deserialization.exception.handler` decides to `CONTINUE` (the default is to `FAIL` and throw an exception). This was the case previously captured in the `skippedDueToDeserializationError` metrics. + * Fetched records having a negative timestamp. + + + +We've also fixed the metrics name for time and session windowed store operations in 2.0. As a result, our current built-in stores will have their store types in the metric names as `in-memory-state`, `in-memory-lru-state`, `rocksdb-state`, `rocksdb-window-state`, and `rocksdb-session-state`. For example, a RocksDB time windowed store's put operation metrics would now be `kafka.streams:type=stream-rocksdb-window-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),rocksdb-window-state-id=([-.\w]+)`. Users need to update their metrics collecting and reporting systems for their time and session windowed stores accordingly. For more details, please read the [State Store Metrics](/37/#kafka_streams_store_monitoring) section. + +We have added support for methods in `ReadOnlyWindowStore` which allows for querying a single window's key-value pair. For users who have customized window store implementations on the above interface, they'd need to update their code to implement the newly added method as well. For more details, see [KIP-261](https://cwiki.apache.org/confluence/display/KAFKA/KIP-261%3A+Add+Single+Value+Fetch+in+Window+Stores). + +We have added public `WindowedSerdes` to allow users to read from / write to a topic storing windowed table changelogs directly. In addition, in `StreamsConfig` we have also added `default.windowed.key.serde.inner` and `default.windowed.value.serde.inner` to let users specify inner serdes if the default serde classes are windowed serdes. For more details, see [KIP-265](https://cwiki.apache.org/confluence/display/KAFKA/KIP-265%3A+Make+Windowed+Serde+to+public+APIs). + +We've added message header support in the `Processor API` in Kafka 2.0.0. In particular, we have added a new API `ProcessorContext#headers()` which returns a `Headers` object that keeps track of the headers of the source topic's message that is being processed. Through this object, users can manipulate the headers map that is being propagated throughout the processor topology as well. For more details please feel free to read the [Developer Guide](/37/streams/developer-guide/processor-api.html#accessing-processor-context) section. + +We have deprecated constructors of `KafkaStreams` that take a `StreamsConfig` as parameter. Please use the other corresponding constructors that accept `java.util.Properties` instead. For more details, see [KIP-245](https://cwiki.apache.org/confluence/display/KAFKA/KIP-245%3A+Use+Properties+instead+of+StreamsConfig+in+KafkaStreams+constructor). + +Kafka 2.0.0 allows to manipulate timestamps of output records using the Processor API ([KIP-251](https://cwiki.apache.org/confluence/display/KAFKA/KIP-251%3A+Allow+timestamp+manipulation+in+Processor+API)). To enable this new feature, `ProcessorContext#forward(...)` was modified. The two existing overloads `#forward(Object key, Object value, String childName)` and `#forward(Object key, Object value, int childIndex)` were deprecated and a new overload `#forward(Object key, Object value, To to)` was added. The new class `To` allows you to send records to all or specific downstream processors by name and to set the timestamp for the output record. Forwarding based on child index is not supported in the new API any longer. + +We have added support to allow routing records dynamically to Kafka topics. More specifically, in both the lower-level `Topology#addSink` and higher-level `KStream#to` APIs, we have added variants that take a `TopicNameExtractor` instance instead of a specific `String` typed topic name, such that for each received record from the upstream processor, the library will dynamically determine which Kafka topic to write to based on the record's key and value, as well as record context. Note that all the Kafka topics that may possibly be used are still considered as user topics and hence required to be pre-created. In addition to that, we have modified the `StreamPartitioner` interface to add the topic name parameter since the topic name now may not be known beforehand; users who have customized implementations of this interface would need to update their code while upgrading their application to use Kafka Streams 2.0.0. + +[KIP-284](https://cwiki.apache.org/confluence/x/DVyHB) changed the retention time for repartition topics by setting its default value to `Long.MAX_VALUE`. Instead of relying on data retention Kafka Streams uses the new purge data API to delete consumed data from those topics and to keep used storage small now. + +We have modified the `ProcessorStateManger#register(...)` signature and removed the deprecated `loggingEnabled` boolean parameter as it is specified in the `StoreBuilder`. Users who used this function to register their state stores into the processor topology need to simply update their code and remove this parameter from the caller. + +Kafka Streams DSL for Scala is a new Kafka Streams client library available for developers authoring Kafka Streams applications in Scala. It wraps core Kafka Streams DSL types to make it easier to call when interoperating with Scala code. For example, it includes higher order functions as parameters for transformations avoiding the need anonymous classes in Java 7 or experimental SAM type conversions in Scala 2.11, automatic conversion between Java and Scala collection types, a way to implicitly provide Serdes to reduce boilerplate from your application and make it more typesafe, and more! For more information see the [Kafka Streams DSL for Scala documentation](/37/streams/developer-guide/dsl-api.html#scala-dsl) and [KIP-270](https://cwiki.apache.org/confluence/display/KAFKA/KIP-270+-+A+Scala+Wrapper+Library+for+Kafka+Streams). + +We have removed these deprecated APIs: + + * `KafkaStreams#toString` no longer returns the topology and runtime metadata; to get topology metadata users can call `Topology#describe()` and to get thread runtime metadata users can call `KafkaStreams#localThreadsMetadata` (they are deprecated since 1.0.0). For detailed guidance on how to update your code please read here + * `TopologyBuilder` and `KStreamBuilder` are removed and replaced by `Topology` and `StreamsBuidler` respectively (they are deprecated since 1.0.0). For detailed guidance on how to update your code please read here + * `StateStoreSupplier` are removed and replaced with `StoreBuilder` (they are deprecated since 1.0.0); and the corresponding `Stores#create` and `KStream, KTable, KGroupedStream` overloaded functions that use it have also been removed. For detailed guidance on how to update your code please read here + * `KStream, KTable, KGroupedStream` overloaded functions that requires serde and other specifications explicitly are removed and replaced with simpler overloaded functions that use `Consumed, Produced, Serialized, Materialized, Joined` (they are deprecated since 1.0.0). For detailed guidance on how to update your code please read here + * `Processor#punctuate`, `ValueTransformer#punctuate`, `ValueTransformer#punctuate` and `ProcessorContext#schedule(long)` are removed and replaced by `ProcessorContext#schedule(long, PunctuationType, Punctuator)` (they are deprecated in 1.0.0). + * The second `boolean` typed parameter "loggingEnabled" in `ProcessorContext#register` has been removed; users can now use `StoreBuilder#withLoggingEnabled, withLoggingDisabled` to specify the behavior when they create the state store. + * `KTable#writeAs, print, foreach, to, through` are removed, users can call `KTable#tostream()#writeAs` instead for the same purpose (they are deprecated since 0.11.0.0). For detailed list of removed APIs please read here + * `StreamsConfig#KEY_SERDE_CLASS_CONFIG, VALUE_SERDE_CLASS_CONFIG, TIMESTAMP_EXTRACTOR_CLASS_CONFIG` are removed and replaced with `StreamsConfig#DEFAULT_KEY_SERDE_CLASS_CONFIG, DEFAULT_VALUE_SERDE_CLASS_CONFIG, DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG` respectively (they are deprecated since 0.11.0.0). + * `StreamsConfig#ZOOKEEPER_CONNECT_CONFIG` are removed as we do not need ZooKeeper dependency in Streams any more (it is deprecated since 0.10.2.0). + + + +## Streams API changes in 1.1.0 + +We have added support for methods in `ReadOnlyWindowStore` which allows for querying `WindowStore`s without the necessity of providing keys. For users who have customized window store implementations on the above interface, they'd need to update their code to implement the newly added method as well. For more details, see [KIP-205](https://cwiki.apache.org/confluence/display/KAFKA/KIP-205%3A+Add+all%28%29+and+range%28%29+API+to+ReadOnlyWindowStore). + +There is a new artifact `kafka-streams-test-utils` providing a `TopologyTestDriver`, `ConsumerRecordFactory`, and `OutputVerifier` class. You can include the new artifact as a regular dependency to your unit tests and use the test driver to test your business logic of your Kafka Streams application. For more details, see [KIP-247](https://cwiki.apache.org/confluence/display/KAFKA/KIP-247%3A+Add+public+test+utils+for+Kafka+Streams). + +The introduction of [KIP-220](https://cwiki.apache.org/confluence/display/KAFKA/KIP-220%3A+Add+AdminClient+into+Kafka+Streams%27+ClientSupplier) enables you to provide configuration parameters for the embedded admin client created by Kafka Streams, similar to the embedded producer and consumer clients. You can provide the configs via `StreamsConfig` by adding the configs with the prefix `admin.` as defined by `StreamsConfig#adminClientPrefix(String)` to distinguish them from configurations of other clients that share the same config names. + +New method in `KTable` + + * `transformValues` methods have been added to `KTable`. Similar to those on `KStream`, these methods allow for richer, stateful, value transformation similar to the Processor API. + + + +New method in `GlobalKTable` + + * A method has been provided such that it will return the store name associated with the `GlobalKTable` or `null` if the store name is non-queryable. + + + +New methods in `KafkaStreams`: + + * added overload for the constructor that allows overriding the `Time` object used for tracking system wall-clock time; this is useful for unit testing your application code. + + + +New methods in `KafkaClientSupplier`: + + * added `getAdminClient(config)` that allows to override an `AdminClient` used for administrative requests such as internal topic creations, etc. + + + +New error handling for exceptions during production: + + * added interface `ProductionExceptionHandler` that allows implementors to decide whether or not Streams should `FAIL` or `CONTINUE` when certain exception occur while trying to produce. + * provided an implementation, `DefaultProductionExceptionHandler` that always fails, preserving the existing behavior by default. + * changing which implementation is used can be done by settings `default.production.exception.handler` to the fully qualified name of a class implementing this interface. + + + +Changes in `StreamsResetter`: + + * added options to specify input topics offsets to reset according to [KIP-171](https://cwiki.apache.org/confluence/display/KAFKA/KIP-171+-+Extend+Consumer+Group+Reset+Offset+for+Stream+Application) + + + +## Streams API changes in 1.0.0 + +With 1.0 a major API refactoring was accomplished and the new API is cleaner and easier to use. This change includes the five main classes `KafkaStreams`, `KStreamBuilder`, `KStream`, `KTable`, and `TopologyBuilder` (and some more others). All changes are fully backward compatible as old API is only deprecated but not removed. We recommend to move to the new API as soon as you can. We will summarize all API changes in the next paragraphs. + +The two main classes to specify a topology via the DSL (`KStreamBuilder`) or the Processor API (`TopologyBuilder`) were deprecated and replaced by `StreamsBuilder` and `Topology` (both new classes are located in package `org.apache.kafka.streams`). Note, that `StreamsBuilder` does not extend `Topology`, i.e., the class hierarchy is different now. The new classes have basically the same methods as the old ones to build a topology via DSL or Processor API. However, some internal methods that were public in `KStreamBuilder` and `TopologyBuilder` but not part of the actual API are not present in the new classes any longer. Furthermore, some overloads were simplified compared to the original classes. See [KIP-120](https://cwiki.apache.org/confluence/display/KAFKA/KIP-120%3A+Cleanup+Kafka+Streams+builder+API) and [KIP-182](https://cwiki.apache.org/confluence/display/KAFKA/KIP-182%3A+Reduce+Streams+DSL+overloads+and+allow+easier+use+of+custom+storage+engines) for full details. + +Changing how a topology is specified also affects `KafkaStreams` constructors, that now only accept a `Topology`. Using the DSL builder class `StreamsBuilder` one can get the constructed `Topology` via `StreamsBuilder#build()`. Additionally, a new class `org.apache.kafka.streams.TopologyDescription` (and some more dependent classes) were added. Those can be used to get a detailed description of the specified topology and can be obtained by calling `Topology#describe()`. An example using this new API is shown in the [quickstart section](/37/streams/quickstart). + +New methods in `KStream`: + + * With the introduction of [KIP-202](https://cwiki.apache.org/confluence/display/KAFKA/KIP-202+Move+merge%28%29+from+StreamsBuilder+to+KStream) a new method `merge()` has been created in `KStream` as the StreamsBuilder class's `StreamsBuilder#merge()` has been removed. The method signature was also changed, too: instead of providing multiple `KStream`s into the method at the once, only a single `KStream` is accepted. + + + +New methods in `KafkaStreams`: + + * retrieve the current runtime information about the local threads via `localThreadsMetadata()` + * observe the restoration of all state stores via `setGlobalStateRestoreListener()`, in which users can provide their customized implementation of the `org.apache.kafka.streams.processor.StateRestoreListener` interface + + + +Deprecated / modified methods in `KafkaStreams`: + + * `toString()`, `toString(final String indent)` were previously used to return static and runtime information. They have been deprecated in favor of using the new classes/methods `localThreadsMetadata()` / `ThreadMetadata` (returning runtime information) and `TopologyDescription` / `Topology#describe()` (returning static information). + * With the introduction of [KIP-182](https://cwiki.apache.org/confluence/display/KAFKA/KIP-182%3A+Reduce+Streams+DSL+overloads+and+allow+easier+use+of+custom+storage+engines) you should no longer pass in `Serde` to `KStream#print` operations. If you can't rely on using `toString` to print your keys an values, you should instead you provide a custom `KeyValueMapper` via the `Printed#withKeyValueMapper` call. + * `setStateListener()` now can only be set before the application start running, i.e. before `KafkaStreams.start()` is called. + + + +Deprecated methods in `KGroupedStream` + + * Windowed aggregations have been deprecated from `KGroupedStream` and moved to `WindowedKStream`. You can now perform a windowed aggregation by, for example, using `KGroupedStream#windowedBy(Windows)#reduce(Reducer)`. + + + +Modified methods in `Processor`: + + * The Processor API was extended to allow users to schedule `punctuate` functions either based on data-driven **stream time** or wall-clock time. As a result, the original `ProcessorContext#schedule` is deprecated with a new overloaded function that accepts a user customizable `Punctuator` callback interface, which triggers its `punctuate` API method periodically based on the `PunctuationType`. The `PunctuationType` determines what notion of time is used for the punctuation scheduling: either [stream time](/37/streams/core-concepts#streams_time) or wall-clock time (by default, **stream time** is configured to represent event time via `TimestampExtractor`). In addition, the `punctuate` function inside `Processor` is also deprecated. + +Before this, users could only schedule based on stream time (i.e. `PunctuationType.STREAM_TIME`) and hence the `punctuate` function was data-driven only because stream time is determined (and advanced forward) by the timestamps derived from the input data. If there is no data arriving at the processor, the stream time would not advance and hence punctuation will not be triggered. On the other hand, When wall-clock time (i.e. `PunctuationType.WALL_CLOCK_TIME`) is used, `punctuate` will be triggered purely based on wall-clock time. So for example if the `Punctuator` function is scheduled based on `PunctuationType.WALL_CLOCK_TIME`, if these 60 records were processed within 20 seconds, `punctuate` would be called 2 times (one time every 10 seconds); if these 60 records were processed within 5 seconds, then no `punctuate` would be called at all. Users can schedule multiple `Punctuator` callbacks with different `PunctuationType`s within the same processor by simply calling `ProcessorContext#schedule` multiple times inside processor's `init()` method. + + + + +If you are monitoring on task level or processor-node / state store level Streams metrics, please note that the metrics sensor name and hierarchy was changed: The task ids, store names and processor names are no longer in the sensor metrics names, but instead are added as tags of the sensors to achieve consistent metrics hierarchy. As a result you may need to make corresponding code changes on your metrics reporting and monitoring tools when upgrading to 1.0.0. Detailed metrics sensor can be found in the [Streams Monitoring](/37/#kafka_streams_monitoring) section. + +The introduction of [KIP-161](https://cwiki.apache.org/confluence/display/KAFKA/KIP-161%3A+streams+deserialization+exception+handlers) enables you to provide a default exception handler for deserialization errors when reading data from Kafka rather than throwing the exception all the way out of your streams application. You can provide the configs via the `StreamsConfig` as `StreamsConfig#DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG`. The specified handler must implement the `org.apache.kafka.streams.errors.DeserializationExceptionHandler` interface. + +The introduction of [KIP-173](https://cwiki.apache.org/confluence/display/KAFKA/KIP-173%3A+Add+prefix+to+StreamsConfig+to+enable+setting+default+internal+topic+configs) enables you to provide topic configuration parameters for any topics created by Kafka Streams. This includes repartition and changelog topics. You can provide the configs via the `StreamsConfig` by adding the configs with the prefix as defined by `StreamsConfig#topicPrefix(String)`. Any properties in the `StreamsConfig` with the prefix will be applied when creating internal topics. Any configs that aren't topic configs will be ignored. If you already use `StateStoreSupplier` or `Materialized` to provide configs for changelogs, then they will take precedence over those supplied in the config. + +## Streams API changes in 0.11.0.0 + +Updates in `StreamsConfig`: + + * new configuration parameter `processing.guarantee` is added + * configuration parameter `key.serde` was deprecated and replaced by `default.key.serde` + * configuration parameter `value.serde` was deprecated and replaced by `default.value.serde` + * configuration parameter `timestamp.extractor` was deprecated and replaced by `default.timestamp.extractor` + * method `keySerde()` was deprecated and replaced by `defaultKeySerde()` + * method `valueSerde()` was deprecated and replaced by `defaultValueSerde()` + * new method `defaultTimestampExtractor()` was added + + + +New methods in `TopologyBuilder`: + + * added overloads for `addSource()` that allow to define a `TimestampExtractor` per source node + * added overloads for `addGlobalStore()` that allow to define a `TimestampExtractor` per source node associated with the global store + + + +New methods in `KStreamBuilder`: + + * added overloads for `stream()` that allow to define a `TimestampExtractor` per input stream + * added overloads for `table()` that allow to define a `TimestampExtractor` per input table + * added overloads for `globalKTable()` that allow to define a `TimestampExtractor` per global table + + + +Deprecated methods in `KTable`: + + * `void foreach(final ForeachAction action)` + * `void print()` + * `void print(final String streamName)` + * `void print(final Serde keySerde, final Serde valSerde)` + * `void print(final Serde keySerde, final Serde valSerde, final String streamName)` + * `void writeAsText(final String filePath)` + * `void writeAsText(final String filePath, final String streamName)` + * `void writeAsText(final String filePath, final Serde keySerde, final Serde valSerde)` + * `void writeAsText(final String filePath, final String streamName, final Serde keySerde, final Serde valSerde)` + + + +The above methods have been deprecated in favor of using the Interactive Queries API. If you want to query the current content of the state store backing the KTable, use the following approach: + + * Make a call to `KafkaStreams.store(final String storeName, final QueryableStoreType queryableStoreType)` + * Then make a call to `ReadOnlyKeyValueStore.all()` to iterate over the keys of a `KTable`. + + + +If you want to view the changelog stream of the `KTable` then you could call `KTable.toStream().print(Printed.toSysOut)`. + +Metrics using exactly-once semantics: + +If `"exactly_once"` processing (EOS version 1) is enabled via the `processing.guarantee` parameter, internally Streams switches from a producer-per-thread to a producer-per-task runtime model. Using `"exactly_once_beta"` (EOS version 2) does use a producer-per-thread, so `client.id` doesn't change, compared with `"at_least_once"` for this case). In order to distinguish the different producers, the producer's `client.id` additionally encodes the task-ID for this case. Because the producer's `client.id` is used to report JMX metrics, it might be required to update tools that receive those metrics. + +Producer's `client.id` naming schema: + + * at-least-once (default): `[client.Id]-StreamThread-[sequence-number]` + * exactly-once: `[client.Id]-StreamThread-[sequence-number]-[taskId]` + * exactly-once-beta: `[client.Id]-StreamThread-[sequence-number]` + + + +`[client.Id]` is either set via Streams configuration parameter `client.id` or defaults to `[application.id]-[processId]` (`[processId]` is a random UUID). + +## Notable changes in 0.10.2.1 + +Parameter updates in `StreamsConfig`: + + * The default config values of embedded producer's `retries` and consumer's `max.poll.interval.ms` have been changed to improve the resiliency of a Kafka Streams application + + + +## Streams API changes in 0.10.2.0 + +New methods in `KafkaStreams`: + + * set a listener to react on application state change via `setStateListener(StateListener listener)` + * retrieve the current application state via `state()` + * retrieve the global metrics registry via `metrics()` + * apply a timeout when closing an application via `close(long timeout, TimeUnit timeUnit)` + * specify a custom indent when retrieving Kafka Streams information via `toString(String indent)` + + + +Parameter updates in `StreamsConfig`: + + * parameter `zookeeper.connect` was deprecated; a Kafka Streams application does no longer interact with ZooKeeper for topic management but uses the new broker admin protocol (cf. [KIP-4, Section "Topic Admin Schema"](https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-TopicAdminSchema.1)) + * added many new parameters for metrics, security, and client configurations + + + +Changes in `StreamsMetrics` interface: + + * removed methods: `addLatencySensor()` + * added methods: `addLatencyAndThroughputSensor()`, `addThroughputSensor()`, `recordThroughput()`, `addSensor()`, `removeSensor()` + + + +New methods in `TopologyBuilder`: + + * added overloads for `addSource()` that allow to define a `auto.offset.reset` policy per source node + * added methods `addGlobalStore()` to add global `StateStore`s + + + +New methods in `KStreamBuilder`: + + * added overloads for `stream()` and `table()` that allow to define a `auto.offset.reset` policy per input stream/table + * added method `globalKTable()` to create a `GlobalKTable` + + + +New joins for `KStream`: + + * added overloads for `join()` to join with `KTable` + * added overloads for `join()` and `leftJoin()` to join with `GlobalKTable` + * note, join semantics in 0.10.2 were improved and thus you might see different result compared to 0.10.0.x and 0.10.1.x (cf. [Kafka Streams Join Semantics](https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+Join+Semantics) in the Apache Kafka wiki) + + +Aligned `null`-key handling for `KTable` joins: + + * like all other KTable operations, `KTable-KTable` joins do not throw an exception on `null` key records anymore, but drop those records silently + + + +New window type _Session Windows_ : + + * added class `SessionWindows` to specify session windows + * added overloads for `KGroupedStream` methods `count()`, `reduce()`, and `aggregate()` to allow session window aggregations + + + +Changes to `TimestampExtractor`: + + * method `extract()` has a second parameter now + * new default timestamp extractor class `FailOnInvalidTimestamp` (it gives the same behavior as old (and removed) default extractor `ConsumerRecordTimestampExtractor`) + * new alternative timestamp extractor classes `LogAndSkipOnInvalidTimestamp` and `UsePreviousTimeOnInvalidTimestamps` + + + +Relaxed type constraints of many DSL interfaces, classes, and methods (cf. [KIP-100](https://cwiki.apache.org/confluence/display/KAFKA/KIP-100+-+Relax+Type+constraints+in+Kafka+Streams+API)). + +## Streams API changes in 0.10.1.0 + +Stream grouping and aggregation split into two methods: + + * old: KStream #aggregateByKey(), #reduceByKey(), and #countByKey() + * new: KStream#groupByKey() plus KGroupedStream #aggregate(), #reduce(), and #count() + * Example: stream.countByKey() changes to stream.groupByKey().count() + + + +Auto Repartitioning: + + * a call to through() after a key-changing operator and before an aggregation/join is no longer required + * Example: stream.selectKey(...).through(...).countByKey() changes to stream.selectKey().groupByKey().count() + + + +TopologyBuilder: + + * methods #sourceTopics(String applicationId) and #topicGroups(String applicationId) got simplified to #sourceTopics() and #topicGroups() + + + +DSL: new parameter to specify state store names: + + * The new Interactive Queries feature requires to specify a store name for all source KTables and window aggregation result KTables (previous parameter "operator/window name" is now the storeName) + * KStreamBuilder#table(String topic) changes to #topic(String topic, String storeName) + * KTable#through(String topic) changes to #through(String topic, String storeName) + * KGroupedStream #aggregate(), #reduce(), and #count() require additional parameter "String storeName" + * Example: stream.countByKey(TimeWindows.of("windowName", 1000)) changes to stream.groupByKey().count(TimeWindows.of(1000), "countStoreName") + + + +Windowing: + + * Windows are not named anymore: TimeWindows.of("name", 1000) changes to TimeWindows.of(1000) (cf. DSL: new parameter to specify state store names) + * JoinWindows has no default size anymore: JoinWindows.of("name").within(1000) changes to JoinWindows.of(1000) + + + +## Streams API broker compatibility + +The following table shows which versions of the Kafka Streams API are compatible with various Kafka broker versions. + +| Kafka Broker (columns) +---|--- +Kafka Streams API (rows) | 0.10.0.x | 0.10.1.x and 0.10.2.x | 0.11.0.x and +1.0.x and +1.1.x and +2.0.x and +2.1.x and +2.2.x and +2.3.x and +2.4.x and +2.5.x and +2.6.x and +2.7.x and +2.8.x and +3.0.x and +3.1.x and +3.2.x and +3.3.x and +3.4.x and +3.5.x and +3.6.x and +3.7.x +0.10.0.x | compatible | compatible | compatible +0.10.1.x and 0.10.2.x | | compatible | compatible +0.11.0.x | | compatible with exactly-once turned off +(requires broker version 0.11.0.x or higher) | compatible +1.0.x and +1.1.x and +2.0.x and +2.1.x and +2.2.0 and +2.2.0 | | compatible with exactly-once turned off +(requires broker version 0.11.0.x or higher); +requires message format 0.10 or higher; +message headers are not supported +(requires broker version 0.11.0.x or higher +with message format 0.11 or higher) | compatible; requires message format 0.10 or higher; +if message headers are used, message format 0.11 +or higher required +2.2.1 and +2.3.x and +2.4.x and +2.5.x and +2.6.x and +2.7.x and +2.8.x and +3.0.x and +3.1.x and +3.2.x and +3.3.x and +3.4.x and +3.5.x and +3.6.x and +3.7.x | | | compatible; requires message format 0.11 or higher; +enabling exactly-once v2 requires 2.4.x or higher + +[Previous](/37/streams/developer-guide/app-reset-tool) Next + + * [Documentation](/documentation) + * [Kafka Streams](/streams) + + diff --git a/docs/toc.html b/docs/toc.html deleted file mode 100644 index 2413491148860..0000000000000 --- a/docs/toc.html +++ /dev/null @@ -1,243 +0,0 @@ - - - - - - -
diff --git a/docs/upgrade.html b/docs/upgrade.html deleted file mode 100644 index c3bee31b10678..0000000000000 --- a/docs/upgrade.html +++ /dev/null @@ -1,2503 +0,0 @@ - - - - - - -
diff --git a/docs/uses.html b/docs/uses.html deleted file mode 100644 index 94d22b477490c..0000000000000 --- a/docs/uses.html +++ /dev/null @@ -1,81 +0,0 @@ - - -

Here is a description of a few of the popular use cases for Apache Kafka®. -For an overview of a number of these areas in action, see this blog post.

- -

Messaging

- -Kafka works well as a replacement for a more traditional message broker. -Message brokers are used for a variety of reasons (to decouple processing from data producers, to buffer unprocessed messages, etc). -In comparison to most messaging systems Kafka has better throughput, built-in partitioning, replication, and fault-tolerance which makes it a good -solution for large scale message processing applications. -

-In our experience messaging uses are often comparatively low-throughput, but may require low end-to-end latency and often depend on the strong -durability guarantees Kafka provides. -

-In this domain Kafka is comparable to traditional messaging systems such as ActiveMQ or -RabbitMQ. - -

Website Activity Tracking

- -The original use case for Kafka was to be able to rebuild a user activity tracking pipeline as a set of real-time publish-subscribe feeds. -This means site activity (page views, searches, or other actions users may take) is published to central topics with one topic per activity type. -These feeds are available for subscription for a range of use cases including real-time processing, real-time monitoring, and loading into Hadoop or -offline data warehousing systems for offline processing and reporting. -

-Activity tracking is often very high volume as many activity messages are generated for each user page view. - -

Metrics

- -Kafka is often used for operational monitoring data. -This involves aggregating statistics from distributed applications to produce centralized feeds of operational data. - -

Log Aggregation

- -Many people use Kafka as a replacement for a log aggregation solution. -Log aggregation typically collects physical log files off servers and puts them in a central place (a file server or HDFS perhaps) for processing. -Kafka abstracts away the details of files and gives a cleaner abstraction of log or event data as a stream of messages. -This allows for lower-latency processing and easier support for multiple data sources and distributed data consumption. - -In comparison to log-centric systems like Scribe or Flume, Kafka offers equally good performance, stronger durability guarantees due to replication, -and much lower end-to-end latency. - -

Stream Processing

- -Many users of Kafka process data in processing pipelines consisting of multiple stages, where raw input data is consumed from Kafka topics and then -aggregated, enriched, or otherwise transformed into new topics for further consumption or follow-up processing. -For example, a processing pipeline for recommending news articles might crawl article content from RSS feeds and publish it to an "articles" topic; -further processing might normalize or deduplicate this content and publish the cleansed article content to a new topic; -a final processing stage might attempt to recommend this content to users. -Such processing pipelines create graphs of real-time data flows based on the individual topics. -Starting in 0.10.0.0, a light-weight but powerful stream processing library called Kafka Streams -is available in Apache Kafka to perform such data processing as described above. -Apart from Kafka Streams, alternative open source stream processing tools include Apache Storm and -Apache Samza. - -

Event Sourcing

- -Event sourcing is a style of application design where state changes are logged as a -time-ordered sequence of records. Kafka's support for very large stored log data makes it an excellent backend for an application built in this style. - -

Commit Log

- -Kafka can serve as a kind of external commit-log for a distributed system. The log helps replicate data between nodes and acts as a re-syncing -mechanism for failed nodes to restore their data. -The log compaction feature in Kafka helps support this usage. -In this usage Kafka is similar to Apache BookKeeper project.