diff --git a/docs/configuration.html b/docs/configuration.html index 7bcb097b944..3f2b0121486 100644 --- a/docs/configuration.html +++ b/docs/configuration.html @@ -43,21 +43,21 @@ 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
+ $ 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
+ $ 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
+ $ 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
+ $ 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
+ $ 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:
@@ -89,7 +89,7 @@
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
+ $ 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
@@ -162,7 +162,7 @@
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
+ $ bin/zookeeper-shell.sh localhost
rmr /controller
Updating Log Cleaner Configs
@@ -220,17 +220,17 @@
3.2 Topic-Level Configs
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 \
+ $ 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
+ $ 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
+ $ 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
+ $ 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.
@@ -294,9 +294,9 @@
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.
-
-Dorg.apache.kafka.disallowed.login.modules=
+ -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
+ -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/connect.html b/docs/connect.html
index 449f066fd0a..ac0c402b1da 100644
--- a/docs/connect.html
+++ b/docs/connect.html
@@ -40,7 +40,7 @@
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 …]
+ $ 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:
@@ -63,7 +63,7 @@
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
+ $ 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.
@@ -118,7 +118,7 @@
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
+ 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:
@@ -129,7 +129,7 @@ value.converter.schemas.enable
After adding the transformations, connect-file-source.properties
file looks as following:
- name=local-file-source
+ name=local-file-source
connector.class=FileStreamSource
tasks.max=1
file=test.txt
@@ -145,13 +145,13 @@ transforms.InsertSource.static.value=test-file-source
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"
+ "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":"foo","data_source":"test-file-source"}
{"line":"bar","data_source":"test-file-source"}
{"line":"hello world","data_source":"test-file-source"}
@@ -208,7 +208,7 @@ transforms.InsertSource.static.value=test-file-source
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
transforms.Filter.type=org.apache.kafka.connect.transforms.Filter
transforms.Filter.predicate=IsFoo
@@ -218,7 +218,7 @@ 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,Extract
transforms.Filter.type=org.apache.kafka.connect.transforms.Filter
transforms.Filter.predicate=IsFoo
@@ -253,7 +253,7 @@ predicates.IsBar.pattern=bar
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
+ 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
.
@@ -315,7 +315,7 @@ predicates.IsBar.pattern=bar
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": {
@@ -328,7 +328,7 @@ predicates.IsBar.pattern=bar
]
}
An example request body for the FileStreamSinkConnector
:
- {
+ {
"offsets": [
{
"partition": {
@@ -370,7 +370,7 @@ predicates.IsBar.pattern=bar
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
+ 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.
@@ -396,7 +396,7 @@ predicates.IsBar.pattern=bar
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
+ # disable retries on failure
errors.retry.timeout=0
# do not log the error and their contexts
@@ -410,7 +410,7 @@ 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
+ # 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
@@ -587,7 +587,7 @@ errors.tolerance=all
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
+ # 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.
@@ -627,7 +627,7 @@ com.example.MySinkConnector
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
+ 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):
@@ -704,7 +704,8 @@ public List<Map<String, String>> taskConfigs(int maxTasks) {
@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.
@@ -754,7 +755,8 @@ public List<SourceRecord> poll() throws InterruptedException {
public abstract void put(Collection<SinkRecord> records);
public void flush(Map<TopicPartition, OffsetAndMetadata> 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.
diff --git a/docs/docker.html b/docs/docker.html
index 1f626ed0607..ef747199e4b 100644
--- a/docs/docker.html
+++ b/docs/docker.html
@@ -26,14 +26,12 @@
Docker is a popular container runtime. Docker images for the JVM based Apache Kafka can be found on Docker Hub and are available from version 3.7.0.
-
- - Docker image can be pulled from Docker Hub using the following command:
-
$ docker pull apache/kafka:{{fullDotVersion}}
- If you want to fetch the latest version of the docker image use following command:
- $ docker pull apache/kafka:latest
- To start the Kafka container using this docker image with default configs and on default port 9092:
- $ docker run -p 9092:9092 apache/kafka:{{fullDotVersion}}
-
+ Docker image can be pulled from Docker Hub using the following command:
+ $ docker pull apache/kafka:{{fullDotVersion}}
+ If you want to fetch the latest version of the Docker image use following command:
+ $ docker pull apache/kafka:latest
+ To start the Kafka container using this Docker image with default configs and on default port 9092:
+ $ docker run -p 9092:9092 apache/kafka:{{fullDotVersion}}
GraalVM Based Native Apache Kafka Docker Image
@@ -42,19 +40,18 @@
NOTE: This image is experimental and intended for local development and testing purposes only; it is not recommended for production use.
-
- - Docker image can be pulled from Docker Hub using the following command:
-
$ docker pull apache/kafka-native:{{fullDotVersion}}
- If you want to fetch the latest version of the docker image use following command:
- $ docker pull apache/kafka-native:latest
- To start the Kafka container using this docker image with default configs and on default port 9092:
- $ docker run -p 9092:9092 apache/kafka-native:{{fullDotVersion}}
-
+ Docker image can be pulled from Docker Hub using the following command:
+ $ docker pull apache/kafka-native:{{fullDotVersion}}
+ If you want to fetch the latest version of the Docker image use following command:
+ $ docker pull apache/kafka-native:latest
+ To start the Kafka container using this Docker image with default configs and on default port 9092:
+ $ docker run -p 9092:9092 apache/kafka-native:{{fullDotVersion}}
+
Usage guide
- Detailed instructions for using the docker image are mentioned here.
+ Detailed instructions for using the Docker image are mentioned here.
diff --git a/docs/implementation.html b/docs/implementation.html
index fb1f52ade5b..78329fa09ff 100644
--- a/docs/implementation.html
+++ b/docs/implementation.html
@@ -32,7 +32,7 @@
5.3.1 Record Batch
The following is the on-disk format of a RecordBatch.
- baseOffset: int64
+ baseOffset: int64
batchLength: int32
partitionLeaderEpoch: int32
magic: int8 (current magic value is 2)
@@ -75,13 +75,13 @@ records: [Record]
5.3.1.1 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)
+ 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.
5.3.2 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
+ length: varint
attributes: int8
bit 0~7: unused
timestampDelta: varlong
@@ -92,7 +92,7 @@ valueLen: varint
value: byte[]
Headers => [Header]
5.3.2.1 Record Header
- headerKeyLength: varint
+ headerKeyLength: varint
headerKey: String
headerValueLength: varint
Value: byte[]
@@ -106,7 +106,7 @@ Value: byte[]
Message Set:
- MessageSet (Version: 0) => [offset message_size message]
+ MessageSet (Version: 0) => [offset message_size message]
offset => INT64
message_size => INT32
message => crc magic_byte attributes key value
@@ -120,7 +120,7 @@ message => crc magic_byte attributes key value
bit 3~7: unused
key => BYTES
value => BYTES
- MessageSet (Version: 1) => [offset message_size message]
+ MessageSet (Version: 1) => [offset message_size message]
offset => INT64
message_size => INT32
message => crc magic_byte attributes timestamp key value
@@ -190,7 +190,7 @@ message => crc magic_byte attributes timestamp key value
The following is the format of the results sent to the consumer.
-
MessageSetSend (fetch result)
+ MessageSetSend (fetch result)
total length : 4 bytes
error code : 2 bytes
@@ -198,7 +198,7 @@ message 1 : x bytes
...
message n : x bytes
- MultiMessageSetSend (multiFetch result)
+ MultiMessageSetSend (multiFetch result)
total length : 4 bytes
error code : 2 bytes
@@ -264,7 +264,7 @@ messageSetSend n
Broker Node Registry
- /brokers/ids/[0...N] --> {"jmx_port":...,"timestamp":...,"endpoints":[...],"host":...,"version":...,"port":...} (ephemeral node)
+ /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.
@@ -272,7 +272,7 @@ messageSetSend n
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)
+ /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.
diff --git a/docs/ops.html b/docs/ops.html
index f161cba58b9..d2b3e96a121 100644
--- a/docs/ops.html
+++ b/docs/ops.html
@@ -28,8 +28,8 @@
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
+ $ bin/kafka-topics.sh --bootstrap-server localhost:9092 --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.
@@ -43,16 +43,16 @@
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
+ $ bin/kafka-topics.sh --bootstrap-server localhost:9092 --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
+ $ bin/kafka-configs.sh --bootstrap-server localhost:9092 --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
+ $ bin/kafka-configs.sh --bootstrap-server localhost:9092 --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
+ $ bin/kafka-topics.sh --bootstrap-server localhost:9092 --delete --topic my_topic_name
Kafka does not currently support reducing the number of partitions for a topic.
@@ -69,7 +69,7 @@
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
+ 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
@@ -78,15 +78,15 @@
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
+ 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
+ $ bin/kafka-leader-election.sh --bootstrap-server localhost:9092 --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
+ 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.
@@ -101,12 +101,11 @@
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
+ $ 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
@@ -114,60 +113,54 @@
For example, to list all consumer groups across all topics:
- > bin/kafka-consumer-groups.sh --bootstrap-server localhost:9092 --list
-
- test-consumer-group
+ $ 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
+ $ 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
+ $ 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 -
+ $ 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
+ $ 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.
+ $ 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 for more details.
+ See KIP-122 for more details.
It has 3 execution options:
@@ -184,7 +177,7 @@
- --reset-offsets also has following scenarios to choose from (at least one scenario must be selected):
+ --reset-offsets also has the following scenarios to choose from (at least one scenario must be selected):
-
--to-datetime <String: datetime> : Reset offsets to offsets from datetime. Format: 'YYYY-MM-DDTHH:mm:SS.sss'
@@ -218,18 +211,12 @@
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
+ $ 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.
@@ -250,61 +237,61 @@
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
- }
+ $ 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
+ $ 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],"log_dirs":["any"]},
+ {"topic":"foo1","partition":1,"replicas":[1,3],"log_dirs":["any"]},
+ {"topic":"foo1","partition":2,"replicas":[3,4],"log_dirs":["any"]},
+ {"topic":"foo2","partition":0,"replicas":[4,2],"log_dirs":["any"]},
+ {"topic":"foo2","partition":1,"replicas":[2,1],"log_dirs":["any"]},
+ {"topic":"foo2","partition":2,"replicas":[1,3],"log_dirs":["any"]}]
+}
- {"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]}]
- }
+Proposed partition reassignment configuration
+{"version":1,
+ "partitions":[{"topic":"foo1","partition":0,"replicas":[6,5],"log_dirs":["any"]},
+ {"topic":"foo1","partition":1,"replicas":[5,6],"log_dirs":["any"]},
+ {"topic":"foo1","partition":2,"replicas":[6,5],"log_dirs":["any"]},
+ {"topic":"foo2","partition":0,"replicas":[5,6],"log_dirs":["any"]},
+ {"topic":"foo2","partition":1,"replicas":[6,5],"log_dirs":["any"]},
+ {"topic":"foo2","partition":2,"replicas":[5,6],"log_dirs":["any"]}]
+}
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
+ $ 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]}]
- }
+{"version":1,
+ "partitions":[{"topic":"foo1","partition":0,"replicas":[2,1],"log_dirs":["any"]},
+ {"topic":"foo1","partition":1,"replicas":[1,3],"log_dirs":["any"]},
+ {"topic":"foo1","partition":2,"replicas":[3,4],"log_dirs":["any"]},
+ {"topic":"foo2","partition":0,"replicas":[4,2],"log_dirs":["any"]},
+ {"topic":"foo2","partition":1,"replicas":[2,1],"log_dirs":["any"]},
+ {"topic":"foo2","partition":2,"replicas":[1,3],"log_dirs":["any"]}]
+}
- 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
-
+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
+ $ 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
@@ -312,26 +299,25 @@
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]}]}
+ $ 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
+ $ 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]}]
- }
+{"version":1,
+ "partitions":[{"topic":"foo1","partition":0,"replicas":[1,2],"log_dirs":["any"]},
+ {"topic":"foo2","partition":1,"replicas":[3,4],"log_dirs":["any"]}]
+}
- Save this to use as the --reassignment-json-file option during rollback
- Successfully started partition reassignments for foo1-0,foo2-1
-
+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
+ $ 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.
@@ -342,27 +328,27 @@
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]}]}
+ $ 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
+ $ 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]}]}
+{"version":1,
+ "partitions":[{"topic":"foo","partition":0,"replicas":[5],"log_dirs":["any"]}]}
- Save this to use as the --reassignment-json-file option during rollback
- Successfully started partition reassignment for foo-0
+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
+ $ 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
+ $ 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.
@@ -372,11 +358,11 @@
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
+ 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
+ $ 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
@@ -384,25 +370,25 @@
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
+ $ 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
+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
+ 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
+ leader.replication.throttled.replicas
+follower.replication.throttled.replicas
Which are configured per topic.
@@ -410,18 +396,18 @@
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
+ $ 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
+ $ 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
@@ -442,8 +428,7 @@
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).
+ 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
@@ -467,48 +452,48 @@
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'.
+ $ 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'.
+ $ 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'.
+ $ 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.
+ $ 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.
+ $ 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.
+ $ 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
+ $ 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
+ $ 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
+ $ 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
+ $ 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
+ $ 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
6.2 Datacenters
@@ -589,15 +574,14 @@
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
+# 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-.*
-
+primary->secondary.topics = foobar-topic, quux-.*
Configuring Geo-Replication
@@ -632,7 +616,7 @@ primary->secondary.topics = foobar-topic, quux-.*
Example: Define MirrorMaker settings (explained in more detail later).
-# Global settings
+# Global settings
clusters = us-west, us-east # defines cluster aliases
us-west.bootstrap.servers = broker3-west:9092
us-east.bootstrap.servers = broker5-east:9092
@@ -641,8 +625,7 @@ 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
-
+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.
@@ -652,9 +635,8 @@ us-west->us.east.topics = foo.*, bar.* # override the default above
Example: Define custom Kafka Connect settings to be used by MirrorMaker.
-# Setting Kafka Connect defaults for MirrorMaker
-tasks.max = 5
-
+# 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.
@@ -668,9 +650,8 @@ tasks.max = 5
Example: Define custom connector settings for the us-west
cluster.
-# us-west custom settings
-us-west.offset.storage.topic = my-mirrormaker-offsets
-
+# 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:
@@ -686,15 +667,14 @@ us-west.offset.storage.topic = my-mirrormaker-offsets
Example: Define custom producer, consumer, admin client settings.
-# us-west cluster (from which to consume)
+# 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
-
+us-east.admin.bootstrap.servers = broker8-secondary:9092
Exactly once
@@ -705,8 +685,7 @@ us-east.admin.bootstrap.servers = broker8-secondary:9092
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
-
+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.
@@ -716,9 +695,8 @@ us-east.admin.bootstrap.servers = broker8-secondary:9092
In either case, it is also necessary to enable intra-cluster communication between the MirrorMaker nodes, as described in KIP-710. 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 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
-
+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 for information on how this can be accomplished.
@@ -728,8 +706,7 @@ listeners = http://localhost:8080
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
-
+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.
@@ -750,18 +727,16 @@ listeners = http://localhost:8080
Example: Define two cluster aliases primary
and secondary
, including their connection information.
-clusters = primary, secondary
+clusters = primary, secondary
primary.bootstrap.servers = broker10-primary:9092,broker-11-primary:9092
-secondary.bootstrap.servers = broker5-secondary:9092,broker6-secondary: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
-
+# 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
.
@@ -793,7 +768,7 @@ The configuration of a replication flow is a combination of top-level default se
Example:
-# Custom top-level defaults that apply to all replication flows
+# Custom top-level defaults that apply to all replication flows
topics = .*
groups = consumer-group1, consumer-group2
@@ -803,8 +778,7 @@ 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
-
+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.
@@ -820,13 +794,12 @@ us-west->us-east.emit.heartbeats = false
Example: Encrypt communication between MirrorMaker and the us-east
cluster.
-us-east.security.protocol=SSL
+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
-
+us-east.ssl.key.password=my-secret-password
Custom Naming of Replicated Topics in Target Clusters
@@ -834,19 +807,17 @@ us-east.ssl.key.password=my-secret-password
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, the names of replicated topics in the target clusters have the format {source}.{source_topic_name}
:
-us-west us-east
+us-west us-east
========= =================
bar-topic
-foo-topic --> us-west.foo-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 = _
-
+# 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.
@@ -862,14 +833,13 @@ us-west->us-east.replication.policy.separator = _
For example, the following two MirrorMaker processes would be racy:
-# Configuration of process 1
+# Configuration of process 1
A->B.enabled = true
A->B.topics = foo
# Configuration of process 2
A->B.enabled = true
-A->B.topics = bar
-
+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.
@@ -885,19 +855,17 @@ A->B.topics = bar
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
+First DC Second DC
========== =========================
primary --------- MirrorMaker --> secondary
-(remote) (local)
-
+(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
-
+# 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.
@@ -907,15 +875,14 @@ The --clusters secondary
tells the MirrorMaker process that the giv
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
+# 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
-
+primary->secondary.topics = foo.* # only replicate some topics
Example: Active/Active High Availability Deployment
@@ -923,14 +890,13 @@ primary->secondary.topics = foo.* # only replicate some topics
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
+# 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
-
+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.
@@ -946,7 +912,7 @@ us-east->us-west.enabled = true
First, define the source and target clusters along with their replication flows in the configuration:
-# Basic settings
+# Basic settings
clusters: west-1, west-2, east-1, east-2, north-1, north-2
west-1.bootstrap.servers = ...
west-2.bootstrap.servers = ...
@@ -973,22 +939,20 @@ 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
-
+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 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
+$ 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
-
+$ 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.
@@ -1008,8 +972,7 @@ $ ./bin/connect-mirror-maker.sh connect-mirror-maker.properties --clusters north
To start a MirrorMaker process, run the command:
-$ ./bin/connect-mirror-maker.sh connect-mirror-maker.properties
-
+$ 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.
@@ -1019,9 +982,9 @@ $ ./bin/connect-mirror-maker.sh connect-mirror-maker.properties --clusters north
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: The cluster alias us-west must be defined in the configuration file
+$ bin/connect-mirror-maker.sh connect-mirror-maker.properties \
+ --clusters us-west
@@ -1034,8 +997,7 @@ $ ./bin/connect-mirror-maker.sh connect-mirror-maker.properties \
You can stop a running MirrorMaker process by sending a SIGTERM signal with the command:
-$ kill <MirrorMaker pid>
-
+$ kill <MirrorMaker pid>
Applying Configuration Changes
@@ -1070,8 +1032,7 @@ $ ./bin/connect-mirror-maker.sh connect-mirror-maker.properties \
The following metrics are emitted:
-# MBean: kafka.connect.mirror:type=MirrorSourceConnector,target=([-.w]+),topic=([-.w]+),partition=([0-9]+)
-
+# 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
@@ -1088,8 +1049,7 @@ byte-rate # average number of bytes/sec in replicated records
checkpoint-latency-ms # time it takes to replicate consumer offsets
checkpoint-latency-ms-min
checkpoint-latency-ms-max
-checkpoint-latency-ms-avg
-
+checkpoint-latency-ms-avg
These metrics do not differentiate between created-at and log-append timestamps.
@@ -1201,13 +1161,12 @@ Security settings for Kafka fall into three main categories, which are similar t
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
+# Grant permissions to user Alice
$ bin/kafka-acls.sh \
- --bootstrap-server broker1:9092 \
+ --bootstrap-server localhost:9092 \
--add --allow-principal User:Alice \
--producer \
- --resource-pattern-type prefixed --topic acme.infosec.
-
+ --resource-pattern-type prefixed --topic acme.infosec.
You can similarly use this approach to isolate different customers on the same shared cluster.
@@ -1270,20 +1229,20 @@ $ bin/kafka-acls.sh \
A Production Server Config
Here is an example production server configuration:
- # ZooKeeper
- zookeeper.connect=[list of ZooKeeper servers]
+ # 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).]
+# 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]
+# 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.
@@ -1301,9 +1260,9 @@ $ bin/kafka-acls.sh \
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
+ -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:
@@ -1366,7 +1325,7 @@ $ bin/kafka-acls.sh \
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
+ $ 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:
@@ -1406,21 +1365,20 @@ $ bin/kafka-acls.sh \
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
- ... ... ... ... ... ...
-
+ $ bin/kafka-metadata-quorum.sh --bootstrap-server localhost:9092 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
+ $ 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
+ $ bin/kafka-server-start.sh server_properties
6.8 Monitoring
@@ -3776,12 +3734,11 @@ for built-in state stores, currently we have:
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
+ 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
+ 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
@@ -3800,7 +3757,7 @@ controller.quorum.voters=1@controller1.example.com:9093,2@controller2.example.co
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
+ $ bin/kafka-metadata-quorum.sh --bootstrap-server localhost:9092 describe --status
ClusterId: fMCL8kv1SWm87L_Md-I2hg
LeaderId: 3002
LeaderEpoch: 2
@@ -3814,18 +3771,17 @@ CurrentObservers: [0,1,2]
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
+ $ 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
+ $ 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
+ $ bin/kafka-metadata-shell.sh --snapshot metadata_log_dir/__cluster_metadata-0/00000000000000000000.log
>> ls /
brokers local metadataQuorum topicIds topics
>> ls /topics
@@ -3842,8 +3798,7 @@ foo
"leaderEpoch" : 0,
"partitionEpoch" : 0
}
->> exit
-
+>> exit
Deploying Considerations
@@ -3909,7 +3864,7 @@ foo
be done by adding the following log4j configuration to each KRaft controller's "log4j.properties" file.
- log4j.logger.org.apache.kafka.metadata.migration=TRACE
+ 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.
@@ -3923,7 +3878,7 @@ foo
of the brokers, or by running the following command.
- ./bin/zookeeper-shell.sh localhost:2181 get /cluster/id
+ $ bin/zookeeper-shell.sh localhost:2181 get /cluster/id
The KRaft controller quorum should also be provisioned with the latest metadata.version
.
@@ -3939,8 +3894,7 @@ foo
Here is a sample config for a KRaft controller that is ready for migration:
-
-# Sample KRaft cluster controller.properties listening on 9093
+ # Sample KRaft cluster controller.properties listening on 9093
process.roles=controller
node.id=3000
controller.quorum.voters=3000@localhost:9093
@@ -3956,7 +3910,7 @@ 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 ...
+# Other configs ...
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.
@@ -3977,8 +3931,7 @@ inter.broker.listener.name=PLAINTEXT
Here is a sample config for a broker that is ready for migration:
-
-# Sample ZK broker server.properties listening on 9092
+ # Sample ZK broker server.properties listening on 9092
broker.id=0
listeners=PLAINTEXT://:9092
advertised.listeners=PLAINTEXT://localhost:9092
@@ -3995,7 +3948,7 @@ zookeeper.connect=localhost:2181
# KRaft controller quorum configuration
controller.quorum.voters=3000@localhost:9093
-controller.listener.names=CONTROLLER
+controller.listener.names=CONTROLLER
Note: Once the final ZK broker has been restarted with the necessary configuration, the migration will automatically begin.
@@ -4025,8 +3978,7 @@ controller.listener.names=CONTROLLER
to use org.apache.kafka.metadata.authorizer.StandardAuthorizer
instead.
-
-# Sample KRaft broker server.properties listening on 9092
+ # Sample KRaft broker server.properties listening on 9092
process.roles=broker
node.id=0
listeners=PLAINTEXT://:9092
@@ -4044,7 +3996,7 @@ listener.security.protocol.map=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT
# Keep the KRaft controller quorum configuration
controller.quorum.voters=3000@localhost:9093
-controller.listener.names=CONTROLLER
+controller.listener.names=CONTROLLER
Each broker is restarted with a KRaft configuration until the entire cluster is running in KRaft mode.
@@ -4061,8 +4013,7 @@ controller.listener.names=CONTROLLER
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
+ # Sample KRaft cluster controller.properties listening on 9093
process.roles=controller
node.id=3000
controller.quorum.voters=3000@localhost:9093
@@ -4075,7 +4026,7 @@ listeners=CONTROLLER://:9093
# Remove ZooKeeper client configuration
# zookeeper.connect=localhost:2181
-# Other configs ...
+# Other configs ...
Reverting to ZooKeeper mode During the Migration
@@ -4274,14 +4225,13 @@ listeners=CONTROLLER://:9093
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
+# please checkout to the specific version tag you're using before building it
# ex: `git checkout {{fullDotVersion}}`
-./gradlew clean :storage:testJar
+$ ./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
+# 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.
@@ -4305,51 +4255,45 @@ rsm.config.dir=/tmp/kafka-remote-storage
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
-
+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
+# 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.
+# 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 \
+$ 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
-
+--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
-
+$ 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
+$ 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
-
+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
+$ 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
+$ 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.
diff --git a/docs/protocol.html b/docs/protocol.html
index 29811a2cdbb..75d8f7e6df1 100644
--- a/docs/protocol.html
+++ b/docs/protocol.html
@@ -182,7 +182,7 @@ Kafka request. SASL/GSSAPI authentication is performed starting with this packet
All requests and responses originate from the following grammar which will be incrementally describe through the rest of this document:
-RequestOrResponse => Size (RequestMessage | ResponseMessage)
+RequestOrResponse => Size (RequestMessage | ResponseMessage)
Size => int32
diff --git a/docs/quickstart.html b/docs/quickstart.html
index f366c1bf2a8..e7dff940046 100644
--- a/docs/quickstart.html
+++ b/docs/quickstart.html
@@ -32,7 +32,7 @@
the latest Kafka release and extract it:
- $ tar -xzf kafka_{{scalaVersion}}-{{fullDotVersion}}.tgz
+ $ tar -xzf kafka_{{scalaVersion}}-{{fullDotVersion}}.tgz
$ cd kafka_{{scalaVersion}}-{{fullDotVersion}}
@@ -42,83 +42,54 @@ $ cd kafka_{{scalaVersion}}-{{fullDotVersion}}
Step 2: Start the Kafka environment
-
- NOTE: Your local environment must have Java 8+ installed.
-
+ 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.
-
+ Apache Kafka can be started using KRaft or ZooKeeper. 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 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
-
+ Generate a Cluster UUID
+ $ KAFKA_CLUSTER_ID="$(bin/kafka-storage.sh random-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
-
- Format Log Directories
-
+ Start the Kafka Server
+ $ bin/kafka-server-start.sh config/kraft/server.properties
- $ bin/kafka-storage.sh format -t $KAFKA_CLUSTER_ID -c config/kraft/server.properties
+ Kafka with ZooKeeper
-
- Start the Kafka Server
-
+ 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
- $ bin/kafka-server-start.sh config/kraft/server.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.
Using JVM Based Apache Kafka Docker Image
-
- - Get the docker image:
-
$ docker pull apache/kafka:{{fullDotVersion}}
- Start the kafka docker container:
- $ docker run -p 9092:9092 apache/kafka:{{fullDotVersion}}
-
+ Get the Docker image:
+ $ docker pull apache/kafka:{{fullDotVersion}}
+
+ Start the Kafka Docker container:
+ $ docker run -p 9092:9092 apache/kafka:{{fullDotVersion}}
Using GraalVM Based Native Apache Kafka Docker Image
-
- - Get the docker image:
-
$ docker pull apache/kafka-native:{{fullDotVersion}}
- Start the kafka docker container:
- $ docker run -p 9092:9092 apache/kafka-native:{{fullDotVersion}}
-
+ Get the Docker image:
+ $ docker pull apache/kafka-native:{{fullDotVersion}}
-
- Once the Kafka server has successfully launched, you will have a basic Kafka environment running and ready to use.
-
+ Start the Kafka Docker container:
+ $ docker run -p 9092:9092 apache/kafka-native:{{fullDotVersion}}
+
+ Once the Kafka server has successfully launched, you will have a basic Kafka environment running and ready to use.
@@ -145,7 +116,7 @@ $ bin/kafka-server-start.sh config/server.properties
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
+ $ 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
@@ -154,7 +125,7 @@ $ bin/kafka-server-start.sh config/server.properties
of the new topic:
- $ bin/kafka-topics.sh --describe --topic quickstart-events --bootstrap-server localhost:9092
+ $ 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
@@ -176,13 +147,11 @@ Topic: quickstart-events Partition: 0 Leader: 0 Replicas: 0 Isr: 0
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
+ $ 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.
-
+ You can stop the producer client with Ctrl-C
at any time.
@@ -193,7 +162,7 @@ Topic: quickstart-events Partition: 0 Leader: 0 Replicas: 0 Isr: 0
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
+ $ bin/kafka-console-consumer.sh --topic quickstart-events --from-beginning --bootstrap-server localhost:9092
This is my first event
This is my second event
@@ -237,16 +206,16 @@ This is my second event
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-{{fullDotVersion}}.jar >> config/connect-standalone.properties"
+ $ echo "plugin.path=libs/connect-file-{{fullDotVersion}}.jar" >> config/connect-standalone.properties
Then, start by creating some seed data to test with:
- $ echo -e "foo\nbar" > test.txt
+ $ echo -e "foo\nbar" > test.txt
Or on Windows:
- $ echo foo> test.txt
-$ echo bar>> test.txt
+ $ 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
@@ -256,7 +225,7 @@ $ echo bar>> test.txt
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
+ $ 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
@@ -273,7 +242,7 @@ $ echo bar>> test.txt
- $ more test.sink.txt
+ $ more test.sink.txt
foo
bar
@@ -283,14 +252,14 @@ bar
- $ bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 --topic connect-test --from-beginning
+ $ 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
+ $ echo "Another line" >> test.txt
You should see the line appear in the console consumer output and in the sink file.
@@ -360,7 +329,7 @@ wordCounts.toStream().to("output-topic", Produced.with(Serdes.String(), Serdes.L
along the way, run the command:
- $ rm -rf /tmp/kafka-logs /tmp/zookeeper /tmp/kraft-combined-logs
+ $ rm -rf /tmp/kafka-logs /tmp/zookeeper /tmp/kraft-combined-logs
diff --git a/docs/security.html b/docs/security.html
index e4bb717c864..6f3f81204a9 100644
--- a/docs/security.html
+++ b/docs/security.html
@@ -50,13 +50,13 @@
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}
+ {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
+ 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
@@ -64,7 +64,7 @@
configuration specifies that the CLIENT
listener will use SSL while the
BROKER
listener will use plaintext.
- listener.security.protocol.map=CLIENT:SSL,BROKER:PLAINTEXT
+ listener.security.protocol.map=CLIENT:SSL,BROKER:PLAINTEXT
Possible options (case-insensitive) for the security protocol are given below:
@@ -82,7 +82,7 @@
we could skip the definition of the CLIENT
and BROKER
listeners
using the following definition:
- listeners=SSL://localhost:9092,PLAINTEXT://localhost:9093
+ 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.
@@ -117,7 +117,7 @@
any security properties that are needed to configure it. For example, we might
use the following configuration on a standalone broker:
- process.roles=broker
+ process.roles=broker
listeners=BROKER://localhost:9092
inter.broker.listener.name=BROKER
controller.quorum.voters=0@localhost:9093
@@ -134,7 +134,7 @@ listener.security.protocol.map=BROKER:SASL_SSL,CONTROLLER:SASL_SSL
is similar. The only difference is that the controller listener must be included in
listeners
:
- process.roles=broker,controller
+ process.roles=broker,controller
listeners=BROKER://localhost:9092,CONTROLLER://localhost:9093
inter.broker.listener.name=BROKER
controller.quorum.voters=0@localhost:9093
@@ -176,7 +176,7 @@ listener.security.protocol.map=BROKER:SASL_SSL,CONTROLLER:SASL_SSL
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
+ $ keytool -keystore {keystorefile} -alias localhost -validity {validity} -genkey -keyalg RSA -storetype pkcs12
You need to specify two parameters in the above command:
- keystorefile: the keystore file that stores the keys (and later the certificate) for this broker. The keystore file contains the private
@@ -192,7 +192,7 @@ listener.security.protocol.map=BROKER:SASL_SSL,CONTROLLER:SASL_SSL
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}
+ $ 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
@@ -205,7 +205,7 @@ listener.security.protocol.map=BROKER:SASL_SSL,CONTROLLER:SASL_SSL
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="
+ $ 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
@@ -228,7 +228,7 @@ listener.security.protocol.map=BROKER:SASL_SSL,CONTROLLER:SASL_SSL
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}
+ $ keytool -keystore server.keystore.jks -alias localhost -validity {validity} -genkey -keyalg RSA -destkeystoretype pkcs12 -ext SAN=DNS:{FQDN},IP:{IPADDRESS1}
Creating your own CA
@@ -252,7 +252,7 @@ listener.security.protocol.map=BROKER:SASL_SSL,CONTROLLER:SASL_SSL
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 = .
+ HOME = .
RANDFILE = $ENV::HOME/.rnd
####################################################################
@@ -337,25 +337,25 @@ 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
+ $ 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
+ $ 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
+ $ 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
+ $ 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
@@ -366,11 +366,11 @@ keyUsage = digitalSignature, keyEncipherment
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}
+ $ 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
+ $ 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:
@@ -439,7 +439,7 @@ keyUsage = digitalSignature, keyEncipherment
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
+ $ openssl x509 -in certificate.crt -text -noout
@@ -447,10 +447,10 @@ keyUsage = digitalSignature, keyEncipherment
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
+ 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.location=/var/private/ssl/server.keystore.jks
ssl.keystore.password=test1234
ssl.key.password=test1234
ssl.truststore.location=/var/private/ssl/server.truststore.jks
@@ -468,7 +468,7 @@ ssl.truststore.password=test1234
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
+ 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 must be obtained and installed in the JDK/JRE. See the
@@ -484,12 +484,12 @@ ssl.truststore.password=test1234
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)
+ 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)
+ $ 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-----
+ -----BEGIN CERTIFICATE-----
{variable sized random bytes}
-----END CERTIFICATE-----
subject=/C=US/ST=CA/L=Santa Clara/O=org/OU=org/CN=Sriharsha Chintalapani
@@ -499,14 +499,14 @@ issuer=/C=US/ST=CA/L=Santa Clara/O=org/OU=org/CN=kafka/emailAddress=test@test.co
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
+ 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.location=/var/private/ssl/client.keystore.jks
ssl.keystore.password=test1234
ssl.key.password=test1234
@@ -520,8 +520,8 @@ ssl.key.password=test1234
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
+ $ bin/kafka-console-producer.sh --bootstrap-server localhost:9093 --topic test --producer.config client-ssl.properties
+$ bin/kafka-console-consumer.sh --bootstrap-server localhost:9093 --topic test --consumer.config client-ssl.properties
7.4 Authentication using SASL
@@ -561,7 +561,7 @@ ssl.key.password=test1234
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 \
+ 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 \
@@ -617,7 +617,7 @@ listener.name.sasl_ssl.plain.sasl.jaas.config=org.apache.kafka.common.security.p
OAUTHBEARER.
For example, GSSAPI
credentials may be configured as:
- KafkaClient {
+ KafkaClient {
com.sun.security.auth.module.Krb5LoginModule required
useKeyTab=true
storeKey=true
@@ -626,7 +626,7 @@ listener.name.sasl_ssl.plain.sasl.jaas.config=org.apache.kafka.common.security.p
};
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
+ -Djava.security.auth.login.config=/etc/kafka/kafka_client_jaas.conf
@@ -695,14 +695,14 @@ listener.name.sasl_ssl.plain.sasl.jaas.config=org.apache.kafka.common.security.p
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}"
+ $ 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}"
Make sure all hosts can be reachable using hostnames - it is a Kerberos requirement that all your hosts can be resolved with their FQDNs.
Configuring Kafka Brokers
- 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 {
+ KafkaServer {
com.sun.security.auth.module.Krb5LoginModule required
useKeyTab=true
storeKey=true
@@ -723,17 +723,17 @@ Client {
allows the broker to login using the keytab specified in this section. See notes for more details on Zookeeper SASL configuration.
- Pass the JAAS and optionally the krb5 file locations as JVM parameters to each Kafka broker (see here for more details):
-
-Djava.security.krb5.conf=/etc/kafka/krb5.conf
+ -Djava.security.krb5.conf=/etc/kafka/krb5.conf
-Djava.security.auth.login.config=/etc/kafka/kafka_server_jaas.conf
- Make sure the keytabs configured in the JAAS file are readable by the operating system user who is starting kafka broker.
- Configure SASL port and SASL mechanisms in server.properties as described here. For example:
-
listeners=SASL_PLAINTEXT://host.name:port
+ 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
+ sasl.kerberos.service.name=kafka
Configuring Kafka Clients
@@ -747,7 +747,7 @@ sasl.enabled.mechanisms=GSSAPI
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 \
+ sasl.jaas.config=com.sun.security.auth.module.Krb5LoginModule required \
useKeyTab=true \
storeKey=true \
keyTab="/etc/security/keytabs/kafka_client.keytab" \
@@ -755,7 +755,7 @@ sasl.enabled.mechanisms=GSSAPI
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 \
+ 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
@@ -763,9 +763,9 @@ sasl.enabled.mechanisms=GSSAPI
KafkaClient
. This option allows only one user for all client connections from a JVM.
Make sure the keytabs configured in the JAAS configuration are readable by the operating system user who is starting kafka client.
Optionally pass the krb5 file locations as JVM parameters to each client JVM (see here for more details):
- -Djava.security.krb5.conf=/etc/kafka/krb5.conf
+ -Djava.security.krb5.conf=/etc/kafka/krb5.conf
Configure the following properties in producer.properties or consumer.properties:
- security.protocol=SASL_PLAINTEXT (or SASL_SSL)
+ security.protocol=SASL_PLAINTEXT (or SASL_SSL)
sasl.mechanism=GSSAPI
sasl.kerberos.service.name=kafka
@@ -781,7 +781,7 @@ sasl.kerberos.service.name=kafka
Configuring Kafka Brokers
- 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 {
+ KafkaServer {
org.apache.kafka.common.security.plain.PlainLoginModule required
username="admin"
password="admin-secret"
@@ -794,9 +794,9 @@ sasl.kerberos.service.name=kafka
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.
Pass the JAAS config file location as JVM parameter to each Kafka broker:
- -Djava.security.auth.login.config=/etc/kafka/kafka_server_jaas.conf
+ -Djava.security.auth.login.config=/etc/kafka/kafka_server_jaas.conf
Configure SASL port and SASL mechanisms in server.properties as described here. For example:
- listeners=SASL_SSL://host.name:port
+ listeners=SASL_SSL://host.name:port
security.inter.broker.protocol=SASL_SSL
sasl.mechanism.inter.broker.protocol=PLAIN
sasl.enabled.mechanisms=PLAIN
@@ -809,7 +809,7 @@ sasl.enabled.mechanisms=PLAIN
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 \
+ 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
@@ -821,7 +821,7 @@ sasl.enabled.mechanisms=PLAIN
as described here. Clients use the login section named
KafkaClient
. This option allows only one user for all client connections from a JVM.
Configure the following properties in producer.properties or consumer.properties:
- security.protocol=SASL_SSL
+ security.protocol=SASL_SSL
sasl.mechanism=PLAIN
@@ -858,21 +858,21 @@ sasl.mechanism=PLAIN
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
+ $ 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 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
+ $ 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
+ $ 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
+ $ 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
Configuring Kafka Brokers
- 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 {
+ KafkaServer {
org.apache.kafka.common.security.scram.ScramLoginModule required
username="admin"
password="admin-secret";
@@ -881,9 +881,9 @@ sasl.mechanism=PLAIN
the broker to initiate connections to other brokers. In this example, admin is the user for
inter-broker communication.
Pass the JAAS config file location as JVM parameter to each Kafka broker:
- -Djava.security.auth.login.config=/etc/kafka/kafka_server_jaas.conf
+ -Djava.security.auth.login.config=/etc/kafka/kafka_server_jaas.conf
Configure SASL port and SASL mechanisms in server.properties as described here. For example:
- listeners=SASL_SSL://host.name:port
+ 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)
@@ -896,7 +896,7 @@ sasl.enabled.mechanisms=SCRAM-SHA-256 (or SCRAM-SHA-512)
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 \
+ sasl.jaas.config=org.apache.kafka.common.security.scram.ScramLoginModule required \
username="alice" \
password="alice-secret";
@@ -909,7 +909,7 @@ sasl.enabled.mechanisms=SCRAM-SHA-256 (or SCRAM-SHA-512)
as described here. Clients use the login section named
KafkaClient
. This option allows only one user for all client connections from a JVM.
Configure the following properties in producer.properties or consumer.properties:
- security.protocol=SASL_SSL
+ security.protocol=SASL_SSL
sasl.mechanism=SCRAM-SHA-256 (or SCRAM-SHA-512)
@@ -944,7 +944,7 @@ sasl.mechanism=SCRAM-SHA-256 (or SCRAM-SHA-512)
Configuring Kafka Brokers
- 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 {
+ KafkaServer {
org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule required
unsecuredLoginStringClaim_sub="admin";
};
@@ -952,9 +952,9 @@ sasl.mechanism=SCRAM-SHA-256 (or SCRAM-SHA-512)
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.
Pass the JAAS config file location as JVM parameter to each Kafka broker:
- -Djava.security.auth.login.config=/etc/kafka/kafka_server_jaas.conf
+ -Djava.security.auth.login.config=/etc/kafka/kafka_server_jaas.conf
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)
+ 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
@@ -967,7 +967,7 @@ sasl.enabled.mechanisms=OAUTHBEARER
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 \
+ sasl.jaas.config=org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule required \
unsecuredLoginStringClaim_sub="alice";
The option unsecuredLoginStringClaim_sub
is used by clients to configure
@@ -980,7 +980,7 @@ sasl.enabled.mechanisms=OAUTHBEARER
as described here. Clients use the login section named
KafkaClient
. This option allows only one user for all client connections from a JVM.
Configure the following properties in producer.properties or consumer.properties:
- security.protocol=SASL_SSL (or SASL_PLAINTEXT if non-production)
+ security.protocol=SASL_SSL (or SASL_PLAINTEXT if non-production)
sasl.mechanism=OAUTHBEARER
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.
@@ -1139,7 +1139,7 @@ sasl.mechanism=OAUTHBEARER
Enabling multiple SASL mechanisms in a broker
- Specify configuration for the login modules of all enabled mechanisms in the
KafkaServer
section of the JAAS config file. For example:
- KafkaServer {
+ KafkaServer {
com.sun.security.auth.module.Krb5LoginModule required
useKeyTab=true
storeKey=true
@@ -1152,9 +1152,9 @@ sasl.mechanism=OAUTHBEARER
user_admin="admin-secret"
user_alice="alice-secret";
};
- Enable the SASL mechanisms in server.properties: sasl.enabled.mechanisms=GSSAPI,PLAIN,SCRAM-SHA-256,SCRAM-SHA-512,OAUTHBEARER
+ Enable the SASL mechanisms in server.properties: sasl.enabled.mechanisms=GSSAPI,PLAIN,SCRAM-SHA-256,SCRAM-SHA-512,OAUTHBEARER
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)
+ security.inter.broker.protocol=SASL_PLAINTEXT (or SASL_SSL)
sasl.mechanism.inter.broker.protocol=GSSAPI (or one of the other enabled mechanisms)
Follow the mechanism-specific steps in GSSAPI (Kerberos),
PLAIN,
@@ -1222,15 +1222,15 @@ sasl.mechanism.inter.broker.protocol=GSSAPI (or one of the other enabled mechani
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
+ $ 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
+ $ 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
+ $ 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
+ $ 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
+ $ bin/kafka-delegation-tokens.sh --bootstrap-server localhost:9092 --describe --command-config client.properties --owner-principal User:user1
Token Authentication
Delegation token authentication piggybacks on the current SASL/SCRAM authentication mechanism. We must enable
@@ -1241,7 +1241,7 @@ sasl.mechanism.inter.broker.protocol=GSSAPI (or one of the other enabled mechani
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 \
+ sasl.jaas.config=org.apache.kafka.common.security.scram.ScramLoginModule required \
username="tokenID123" \
password="lAYYSFmLs4bTjf+lTZ1LCHR/ZZFNA==" \
tokenauth="true";
@@ -1278,9 +1278,9 @@ sasl.mechanism.inter.broker.protocol=GSSAPI (or one of the other enabled mechani
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
+ 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
+ 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 and
@@ -1288,9 +1288,9 @@ sasl.mechanism.inter.broker.protocol=GSSAPI (or one of the other enabled mechani
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
+ 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
+ super.users=User:Bob;User:Alice
KRaft Principal Forwarding
@@ -1314,11 +1314,11 @@ sasl.mechanism.inter.broker.protocol=GSSAPI (or one of the other enabled mechani
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/
RULE:pattern/replacement/[LU]
Example ssl.principal.mapping.rules
values are:
- RULE:^CN=(.*?),OU=ServiceUsers.*$/$1/,
+ 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
@@ -1327,14 +1327,14 @@ DEFAULT
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
+ 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). 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/
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
@@ -1342,7 +1342,7 @@ 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
+ 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:
@@ -1545,41 +1545,41 @@ RULE:[n:string](regexp)s/pattern/replacement/g/U
- 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
+ $ 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
+ $ 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 '*'
+ $ 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
+ $ 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
+ $ 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
+
$ 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
+ $ 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 '*'
+ $ 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
+ > 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
+ $ 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
+ $ 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.
@@ -1587,10 +1587,10 @@ RULE:[n:string](regexp)s/pattern/replacement/g/U
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"
+ $ 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"
@@ -2356,42 +2356,42 @@ bin/kafka-acls.sh --bootstrap-server localhost:9092 --command-config /tmp/adminc
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
+ 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,...]
+ 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
+ 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
+ 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
+ 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,...]
+ 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
+ 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
+ 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.
@@ -2455,7 +2455,7 @@ security.inter.broker.protocol=SSL
Here is a sample (partial) ZooKeeper configuration for enabling TLS authentication.
These configurations are described in the
ZooKeeper Admin Guide.
- secureClientPort=2182
+ secureClientPort=2182
serverCnxnFactory=org.apache.zookeeper.server.NettyServerCnxnFactory
authProvider.x509=org.apache.zookeeper.server.auth.X509AuthenticationProvider
ssl.keyStore.location=/path/to/zk/keystore.jks
@@ -2469,7 +2469,7 @@ ssl.trustStore.password=zk-ts-passwd
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
+ # 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
@@ -2490,7 +2490,7 @@ zookeeper.set.acl=true
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:
- 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
+ clientPort=2181
secureClientPort=2182
serverCnxnFactory=org.apache.zookeeper.server.NettyServerCnxnFactory
authProvider.x509=org.apache.zookeeper.server.auth.X509AuthenticationProvider
@@ -2513,9 +2513,9 @@ ssl.trustStore.password=zk-ts-passwd
- 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
+ $ 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
+ $ bin/zookeeper-security-migration.sh --help
7.7.3 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:
@@ -2533,7 +2533,7 @@ ssl.trustStore.password=zk-ts-passwd
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
+ # 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
diff --git a/docs/streams/developer-guide/app-reset-tool.html b/docs/streams/developer-guide/app-reset-tool.html
index a877b54f36e..0951b3b4f61 100644
--- a/docs/streams/developer-guide/app-reset-tool.html
+++ b/docs/streams/developer-guide/app-reset-tool.html
@@ -78,9 +78,9 @@
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.
- <path-to-kafka>/bin/kafka-streams-application-reset
+ $ bin/kafka-streams-application-reset
The tool accepts the following parameters:
- Option (* = required) Description
+ Option (* = required) Description
--------------------- -----------
* --application-id <String: id> The Kafka Streams application ID
(application.id).
diff --git a/docs/streams/developer-guide/datatypes.html b/docs/streams/developer-guide/datatypes.html
index 607d133f3b2..afa6397c5ef 100644
--- a/docs/streams/developer-guide/datatypes.html
+++ b/docs/streams/developer-guide/datatypes.html
@@ -100,7 +100,7 @@ userCountByRegion.to("RegionCountsTopic", Produced.valueSerde(Serdes.L
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:
- <dependency>
+ <dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>kafka-clients</artifactId>
<version>2.8.0</version>
diff --git a/docs/streams/developer-guide/dsl-topology-naming.html b/docs/streams/developer-guide/dsl-topology-naming.html
index 72e0c2887a1..ec3bc857c10 100644
--- a/docs/streams/developer-guide/dsl-topology-naming.html
+++ b/docs/streams/developer-guide/dsl-topology-naming.html
@@ -81,7 +81,7 @@ stream.filter((k,v) -> !v.equals("invalid_txn"))
Running Topology#describe()
yields this string:
-
Topologies:
+ Topologies:
Sub-topology: 0
Source: KSTREAM-SOURCE-0000000000 (topics: [input])
--> KSTREAM-FILTER-0000000001
diff --git a/docs/streams/developer-guide/memory-mgmt.html b/docs/streams/developer-guide/memory-mgmt.html
index 7f5bc09cef5..591b97bb180 100644
--- a/docs/streams/developer-guide/memory-mgmt.html
+++ b/docs/streams/developer-guide/memory-mgmt.html
@@ -159,11 +159,10 @@ props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
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)
+ # 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”
-
+$ 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 and count its memory against the block cache, and then pass the same Cache object to each instance. See RocksDB Memory Usage for details. An example RocksDBConfigSetter implementing this is shown below:
public static class BoundedMemoryRocksDBConfig implements RocksDBConfigSetter {
diff --git a/docs/streams/developer-guide/running-app.html b/docs/streams/developer-guide/running-app.html
index ff3ed75d290..7ef2b98a580 100644
--- a/docs/streams/developer-guide/running-app.html
+++ b/docs/streams/developer-guide/running-app.html
@@ -51,7 +51,7 @@
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`
+ # 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
diff --git a/docs/streams/developer-guide/security.html b/docs/streams/developer-guide/security.html
index 74319f9e33c..e3622bf80e8 100644
--- a/docs/streams/developer-guide/security.html
+++ b/docs/streams/developer-guide/security.html
@@ -98,7 +98,7 @@
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
+ # 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
diff --git a/docs/streams/developer-guide/testing.html b/docs/streams/developer-guide/testing.html
index b5fadb12b3d..2c70a402459 100644
--- a/docs/streams/developer-guide/testing.html
+++ b/docs/streams/developer-guide/testing.html
@@ -51,7 +51,7 @@
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:
- <dependency>
+ <dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>kafka-streams-test-utils</artifactId>
<version>{{fullDotVersion}}</version>
diff --git a/docs/streams/quickstart.html b/docs/streams/quickstart.html
index 0814723523c..87ec24b3a20 100644
--- a/docs/streams/quickstart.html
+++ b/docs/streams/quickstart.html
@@ -91,8 +91,8 @@ because it cannot know when it has processed "all" the input data.
Download the {{fullDotVersion}} release and un-tar it.
Note that there are multiple downloadable Scala versions and we choose to use the recommended version ({{scalaVersion}}) here:
-> tar -xzf kafka_{{scalaVersion}}-{{fullDotVersion}}.tgz
-> cd kafka_{{scalaVersion}}-{{fullDotVersion}}
+$ tar -xzf kafka_{{scalaVersion}}-{{fullDotVersion}}.tgz
+$ cd kafka_{{scalaVersion}}-{{fullDotVersion}}
Step 2: Start the Kafka server
@@ -108,13 +108,13 @@ Note that there are multiple downloadable Scala versions and we choose to use th
Run the following commands in order to start all services in the correct order:
-> bin/zookeeper-server-start.sh config/zookeeper.properties
+$ bin/zookeeper-server-start.sh config/zookeeper.properties
Open another terminal session and run:
-> bin/kafka-server-start.sh config/server.properties
+$ bin/kafka-server-start.sh config/server.properties
Kafka with KRaft
@@ -124,19 +124,19 @@ Note that there are multiple downloadable Scala versions and we choose to use th
Generate a Cluster UUID
-> KAFKA_CLUSTER_ID="$(bin/kafka-storage.sh random-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
+$ 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
+$ bin/kafka-server-start.sh config/kraft/server.properties
Step 3: Prepare input topic and start Kafka producer
@@ -152,7 +152,7 @@ Or on Windows:
Next, we create the input topic named streams-plaintext-input and the output topic named streams-wordcount-output:
-> bin/kafka-topics.sh --create \
+$ bin/kafka-topics.sh --create \
--bootstrap-server localhost:9092 \
--replication-factor 1 \
--partitions 1 \
@@ -162,7 +162,7 @@ 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 \
+$ bin/kafka-topics.sh --create \
--bootstrap-server localhost:9092 \
--replication-factor 1 \
--partitions 1 \
@@ -172,8 +172,7 @@ 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
-
+$ 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
@@ -183,7 +182,7 @@ Topic:streams-plaintext-input PartitionCount:1 ReplicationFactor:1 Configs:segme
The following command starts the WordCount demo application:
-> bin/kafka-run-class.sh org.apache.kafka.streams.examples.wordcount.WordCountDemo
+$ 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,
@@ -193,11 +192,11 @@ Hence there won't be any STDOUT output except log entries as the results are wri
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
+$ 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 \
+$ bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 \
--topic streams-wordcount-output \
--from-beginning \
--property print.key=true \
@@ -212,14 +211,14 @@ Now let's write some message with the console producer into the input topic s
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
+$ 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 \
+$ bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 \
--topic streams-wordcount-output \
--from-beginning \
--property print.key=true \
@@ -241,13 +240,13 @@ Now let's continue writing one more message with the console producer into the i
Enter the text line "hello kafka streams" and hit <RETURN>.
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
+$ 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 \
+$ bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 \
--topic streams-wordcount-output \
--from-beginning \
--property print.key=true \
@@ -269,15 +268,15 @@ Whenever you write further input messages to the input topic, you will observe n
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 <RETURN> 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
+$ 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 \
+$ bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 \
--topic streams-wordcount-output \
--from-beginning \
--property print.key=true \
diff --git a/docs/streams/tutorial.html b/docs/streams/tutorial.html
index efa6eba6933..933cd11f12b 100644
--- a/docs/streams/tutorial.html
+++ b/docs/streams/tutorial.html
@@ -42,31 +42,31 @@
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={{fullDotVersion}} \
- -DgroupId=streams.examples \
- -DartifactId=streams-quickstart\
- -Dversion=0.1 \
- -Dpackage=myapps
+ $ mvn archetype:generate \
+-DarchetypeGroupId=org.apache.kafka \
+-DarchetypeArtifactId=streams-quickstart-java \
+-DarchetypeVersion={{fullDotVersion}} \
+-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
+ $ 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.
@@ -78,8 +78,8 @@
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
+ $ cd streams-quickstart
+$ rm src/main/java/myapps/*.java
Writing a first Streams application: Pipe
@@ -165,8 +165,8 @@ props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getCla
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
+ $ 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
@@ -273,8 +273,8 @@ public class Pipe {
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
+ $ 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,
@@ -290,7 +290,7 @@ public class Pipe {
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
+ $ 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:
@@ -342,8 +342,8 @@ source.flatMapValues(value -> Arrays.asList(value.split("\\W+")))
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
+ $ 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
@@ -482,8 +482,8 @@ source.flatMapValues(value -> Arrays.asList(value.toLowerCase(Locale.getDefault(
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
+ $ 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
diff --git a/docs/upgrade.html b/docs/upgrade.html
index a16ac8c0e92..afd092a4c6a 100644
--- a/docs/upgrade.html
+++ b/docs/upgrade.html
@@ -86,7 +86,7 @@
Once the cluster's behavior and performance has been verified, bump the metadata.version by running
- ./bin/kafka-features.sh upgrade --metadata 3.7
+ bin/kafka-features.sh upgrade --metadata 3.7
Note that cluster metadata downgrade is not supported in this version since it has metadata changes.
@@ -192,7 +192,7 @@
Once the cluster's behavior and performance has been verified, bump the metadata.version by running
- ./bin/kafka-features.sh upgrade --metadata 3.6
+ bin/kafka-features.sh upgrade --metadata 3.6
Note that cluster metadata downgrade is not supported in this version since it has metadata changes.
@@ -334,7 +334,7 @@
Once the cluster's behavior and performance has been verified, bump the metadata.version by running
- ./bin/kafka-features.sh upgrade --metadata 3.5
+ bin/kafka-features.sh upgrade --metadata 3.5
Note that cluster metadata downgrade is not supported in this version since it has metadata changes.
@@ -433,7 +433,7 @@
Once the cluster's behavior and performance has been verified, bump the metadata.version by running
- ./bin/kafka-features.sh upgrade --metadata 3.4
+ bin/kafka-features.sh upgrade --metadata 3.4
Note that cluster metadata downgrade is not supported in this version since it has metadata changes.
@@ -503,7 +503,7 @@
Once the cluster's behavior and performance has been verified, bump the metadata.version by running
- ./bin/kafka-features.sh upgrade --metadata 3.3
+ bin/kafka-features.sh upgrade --metadata 3.3
Note that cluster metadata downgrade is not supported in this version since it has metadata changes.
@@ -596,7 +596,7 @@
possible compatibility issues originating from the logging framework.
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
.
+ 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