mirror of https://github.com/apache/kafka.git
				
				
				
			
		
			
				
	
	
		
			321 lines
		
	
	
		
			19 KiB
		
	
	
	
		
			HTML
		
	
	
	
			
		
		
	
	
			321 lines
		
	
	
		
			19 KiB
		
	
	
	
		
			HTML
		
	
	
	
| <!--
 | ||
|  Licensed to the Apache Software Foundation (ASF) under one or more
 | ||
|  contributor license agreements.  See the NOTICE file distributed with
 | ||
|  this work for additional information regarding copyright ownership.
 | ||
|  The ASF licenses this file to You under the Apache License, Version 2.0
 | ||
|  (the "License"); you may not use this file except in compliance with
 | ||
|  the License.  You may obtain a copy of the License at
 | ||
| 
 | ||
|     http://www.apache.org/licenses/LICENSE-2.0
 | ||
| 
 | ||
|  Unless required by applicable law or agreed to in writing, software
 | ||
|  distributed under the License is distributed on an "AS IS" BASIS,
 | ||
|  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 | ||
|  See the License for the specific language governing permissions and
 | ||
|  limitations under the License.
 | ||
| -->
 | ||
| 
 | ||
| <!--#include virtual="../../includes/_header.htm" -->
 | ||
| <!--#include virtual="../../includes/_top.htm" -->
 | ||
| <div class="content">
 | ||
| <div class="p-zk2kraft">
 | ||
|     <h1>Differences Between KRaft mode and ZooKeeper mode</h1>
 | ||
|     <h2 class="anchor-heading">Removed ZooKeeper Features</h2>
 | ||
|     <p>
 | ||
|         This section documents differences in behavior between KRaft mode and ZooKeeper mode.
 | ||
|         Specifically, several configurations, metrics and features have changed or are no longer required in KRaft mode.
 | ||
|         To migrate an existing cluster from ZooKeeper mode to KRaft mode, please refer to the
 | ||
|         <a href="/39/documentation.html#kraft_zk_migration">ZooKeeper to KRaft Migration</a> section.
 | ||
|     </p>
 | ||
|     <h3 class="anchor-heading">Configurations</h3>
 | ||
|     <ul>
 | ||
|         <li>
 | ||
|             <p>
 | ||
|                 Removed password encoder-related configurations. These configurations were used in
 | ||
|                 ZooKeeper mode to define the key and backup key for encrypting sensitive data (e.g., passwords),
 | ||
|                 specify the algorithm and key generation method for password encryption (e.g., AES, RSA), and control
 | ||
|                 the key length and encryption strength.
 | ||
|             </p>
 | ||
|             <ul>
 | ||
|                 <li><code>password.encoder.secret</code></li>
 | ||
|                 <li><code>password.encoder.old.secret</code></li>
 | ||
|                 <li><code>password.encoder.keyfactory.algorithm</code></li>
 | ||
|                 <li><code>password.encoder.cipher.algorithm</code></li>
 | ||
|                 <li><code>password.encoder.key.length</code></li>
 | ||
|                 <li><code>password.encoder.iterations</code></li>
 | ||
|             </ul>
 | ||
|             <p>
 | ||
|                 In KRaft mode, Kafka stores sensitive data in records, and the data is not encrypted in Kafka.
 | ||
|             </p>
 | ||
|         </li>
 | ||
|         <li>
 | ||
|             <p>
 | ||
|                 Removed <code>control.plane.listener.name</code>. Kafka relies on ZooKeeper to manage metadata, but some
 | ||
|                 internal operations (e.g., communication between controllers (a.k.a., broker controller) and brokers)
 | ||
|                 still require Kafka’s internal control plane for coordination.
 | ||
|             </p>
 | ||
|             <p>
 | ||
|                 In KRaft mode, Kafka eliminates its dependency on ZooKeeper, and the control plane functionality is
 | ||
|                 fully integrated into Kafka itself. The process roles are clearly separated: brokers handle data-related
 | ||
|                 requests, while the controllers (a.k.a., quorum controller) manages metadata-related requests. The controllers
 | ||
|                 use the Raft protocol for internal communication, which operates differently from the ZooKeeper model. Use the
 | ||
|                 following parameters to configure the control plane listener:
 | ||
|             </p>
 | ||
|             <ul>
 | ||
|                 <li><code>controller.listener.names</code></li>
 | ||
|                 <li><code>listeners</code></li>
 | ||
|                 <li><code>listener.security.protocol.map</code></li>
 | ||
|             </ul>
 | ||
|         </li>
 | ||
|         <li>
 | ||
|             <p>
 | ||
|                 Removed graceful broker shutdowns-related configurations. These configurations were used in ZooKeeper mode
 | ||
|                 to define the maximum number of retries and the retry backoff time for controlled shutdowns. It can
 | ||
|                 reduce the risk of unplanned leader changes and data inconsistencies.
 | ||
|             </p>
 | ||
|             <ul>
 | ||
|                 <li><code>controlled.shutdown.max.retries</code></li>
 | ||
|                 <li><code>controlled.shutdown.retry.backoff.ms</code></li>
 | ||
|             </ul>
 | ||
|             <p>
 | ||
|                 In KRaft mode, Kafka uses the Raft protocol to manage metadata. The broker shutdown process differs from
 | ||
|                 ZooKeeper mode as it is managed by the quorum-based controller. The shutdown process is more reliable
 | ||
|                 and efficient due to automated leader transfers and metadata updates handled by the controller.
 | ||
|             </p>
 | ||
|         </li>
 | ||
|         <li>
 | ||
|             <p>
 | ||
|                 Removed the broker id generation-related configurations. These configurations were used in ZooKeeper mode
 | ||
|                 to specify the broker id auto generation and control the broker id generation process.
 | ||
|             </p>
 | ||
|             <ul>
 | ||
|                 <li><code>reserved.broker.max.id</code></li>
 | ||
|                 <li><code>broker.id.generation.enable</code></li>
 | ||
|             </ul>
 | ||
|             <p>
 | ||
|                 Kafka uses the node id in KRaft mode to identify servers.
 | ||
|             </p>
 | ||
|             <ul>
 | ||
|                 <li><code>node.id</code></li>
 | ||
|             </ul>
 | ||
|         </li>
 | ||
|         <li>
 | ||
|             <p>
 | ||
|                 Removed broker protocol version-related configurations. These configurations were used in ZooKeeper mode to define communication protocol version between brokers.
 | ||
|                 In KRaft mode, Kafka uses <code>metadata.version</code> to control the feature level of the cluster, which can be managed using <code>bin/kafka-features.sh</code>.
 | ||
|             </p>
 | ||
|             <ul>
 | ||
|                 <li><code>inter.broker.protocol.version</code></li>
 | ||
|             </ul>
 | ||
|         </li>
 | ||
|         <li>
 | ||
|             <p>
 | ||
|                 Removed dynamic configurations which relied on ZooKeeper. In KRaft mode, to change these configurations,
 | ||
|                 you need to restart the broker/controller.
 | ||
|             </p>
 | ||
|             <ul>
 | ||
|                 <li><code>advertised.listeners</code></li>
 | ||
|             </ul>
 | ||
|         </li>
 | ||
|         <li>
 | ||
|             <p>
 | ||
|                 Removed the leader imbalance configuration used only in ZooKeeper. <code>leader.imbalance.per.broker.percentage</code>
 | ||
|                 was used to limit the preferred leader election frequency in ZooKeeper.
 | ||
|             </p>
 | ||
|             <ul>
 | ||
|                 <li><code>leader.imbalance.per.broker.percentage</code></li>
 | ||
|             </ul>
 | ||
|         </li>
 | ||
|         <li>
 | ||
|             <p>
 | ||
|                 Removed ZooKeeper related configurations.
 | ||
|             </p>
 | ||
|             <ul>
 | ||
|                 <li><code>zookeeper.connect</code></li>
 | ||
|                 <li><code>zookeeper.session.timeout.ms</code></li>
 | ||
|                 <li><code>zookeeper.connection.timeout.ms</code></li>
 | ||
|                 <li><code>zookeeper.set.acl</code></li>
 | ||
|                 <li><code>zookeeper.max.in.flight.requests</code></li>
 | ||
|                 <li><code>zookeeper.ssl.client.enable</code></li>
 | ||
|                 <li><code>zookeeper.clientCnxnSocket</code></li>
 | ||
|                 <li><code>zookeeper.ssl.keystore.location</code></li>
 | ||
|                 <li><code>zookeeper.ssl.keystore.password</code></li>
 | ||
|                 <li><code>zookeeper.ssl.keystore.type</code></li>
 | ||
|                 <li><code>zookeeper.ssl.truststore.location</code></li>
 | ||
|                 <li><code>zookeeper.ssl.truststore.password</code></li>
 | ||
|                 <li><code>zookeeper.ssl.truststore.type</code></li>
 | ||
|                 <li><code>zookeeper.ssl.protocol</code></li>
 | ||
|                 <li><code>zookeeper.ssl.enabled.protocols</code></li>
 | ||
|                 <li><code>zookeeper.ssl.cipher.suites</code></li>
 | ||
|                 <li><code>zookeeper.ssl.endpoint.identification.algorithm</code></li>
 | ||
|                 <li><code>zookeeper.ssl.crl.enable</code></li>
 | ||
|                 <li><code>zookeeper.ssl.ocsp.enable</code></li>
 | ||
|             </ul>
 | ||
|         </li>
 | ||
|     </ul>
 | ||
|     <h3 class="anchor-heading">Dynamic Log Levels</h3>
 | ||
|     <ul>
 | ||
|         <li>
 | ||
|             <p>
 | ||
|                 The dynamic log levels feature allows you to change the log4j settings of a running broker or controller process without restarting it. The command-line syntax for setting dynamic log levels on brokers has not changed in KRaft mode. Here is an example of setting the log level on a broker:<br/>
 | ||
|                 <pre><code class="language-bash">
 | ||
| ./bin/kafka-configs.sh --bootstrap-server localhost:9092 \
 | ||
|     --entity-type broker-loggers \
 | ||
|     --entity-name 1 \
 | ||
|     --alter \
 | ||
|     --add-config org.apache.kafka.raft.KafkaNetworkChannel=TRACE
 | ||
|                 </code></pre>
 | ||
|             </p>
 | ||
|         </li>
 | ||
|         <li>
 | ||
|             <p>
 | ||
|                 When setting dynamic log levels on the controllers, the <code>--bootstrap-controller</code> flag must be used. Here is an example of setting the log level ona  controller:<br/>
 | ||
|                 <pre><code class="language-bash">
 | ||
| ./bin/kafka-configs.sh --bootstrap-controller localhost:9093 \
 | ||
|     --entity-type broker-loggers \
 | ||
|     --entity-name 1 \
 | ||
|     --alter \
 | ||
|     --add-config org.apache.kafka.raft.KafkaNetworkChannel=TRACE
 | ||
|                 </code></pre><br/>
 | ||
|                 Note that the entity-type must be specified as <code>broker-loggers</code>, even though we are changing a controller's log level rather than a broker's log level.
 | ||
|             </p>
 | ||
|         </li>
 | ||
|         <li>
 | ||
|             <p>
 | ||
|                 When changing the log level of a combined node, which has both broker and controller roles, either --bootstrap-servers or --bootstrap-controllers may be used. Combined nodes have only a single set of log levels; there are not different log levels for the broker and controller parts of the process.
 | ||
|             </p>
 | ||
|         </li>
 | ||
|     </ul>
 | ||
|     <h3 class="anchor-heading">Dynamic Controller Configurations</h3>
 | ||
|     <ul>
 | ||
|         <li>
 | ||
|             <p>
 | ||
|                 Some Kafka configurations can be changed dynamically, without restarting the process.  The command-line syntax for setting dynamic log levels on brokers has not changed in KRaft mode. Here is an example of setting the number of IO threads on a broker:<br/>
 | ||
|                 <pre><code class="language-bash">
 | ||
| ./bin/kafka-configs.sh --bootstrap-server localhost:9092 \
 | ||
|     --entity-type brokers \
 | ||
|     --entity-name 1 \
 | ||
|     --alter \
 | ||
|     --add-config num.io.threads=5
 | ||
|                 </code></pre>
 | ||
|             </p>
 | ||
|         </li>
 | ||
|         <li>
 | ||
|             <p>
 | ||
|                 Controllers will apply all applicable cluster-level dynamic configurations. For example, the following command-line will change the <code>max.connections</code> setting on all of the brokers and all of the controllers in the cluster:<br/>
 | ||
|                 <pre><code class="language-bash">
 | ||
| ./bin/kafka-configs.sh --bootstrap-server localhost:9092 \
 | ||
|     --entity-type brokers \
 | ||
|     --entity-default \
 | ||
|     --alter \
 | ||
|     --add-config max.connections=10000
 | ||
|                 </code></pre><br/>
 | ||
|                 It is not currently possible to apply a dynamic configuration on only a single controller.
 | ||
|             </p>
 | ||
|         </li>
 | ||
|     </ul>
 | ||
|     <h2 class="anchor-heading">Metrics</h2>
 | ||
|     <ul>
 | ||
|         <li>
 | ||
|             <p>
 | ||
|                 Removed the following metrics related to ZooKeeper.
 | ||
|                 <code>ControlPlaneNetworkProcessorAvgIdlePercent</code>
 | ||
|                 is to monitor the average fraction of time the network processors are idle. The other <code>ControlPlaneExpiredConnectionsKilledCount</code>
 | ||
|                 is to monitor the total number of connections disconnected, across all processors.
 | ||
|             </p>
 | ||
|             <ul>
 | ||
|                 <li><code>ControlPlaneNetworkProcessorAvgIdlePercent</code></li>
 | ||
|                 <li><code>ControlPlaneExpiredConnectionsKilledCount</code></li>
 | ||
|             </ul>
 | ||
|             <p>
 | ||
|                 In KRaft mode, Kafka also provides metrics to monitor the network processors and expired connections.
 | ||
|                 Use the following metrics to monitor the network processors and expired connections:
 | ||
|             </p>
 | ||
|             <ul>
 | ||
|                 <li><code>NetworkProcessorAvgIdlePercent</code></li>
 | ||
|                 <li><code>ExpiredConnectionsKilledCount</code></li>
 | ||
|             </ul>
 | ||
|         </li>
 | ||
|         <li>
 | ||
|             <p>
 | ||
|                 Removed the metrics which are only used in ZooKeeper mode.
 | ||
|             </p>
 | ||
|             <ul>
 | ||
|                 <li><code>kafka.controller:type=ControllerChannelManager,name=QueueSize</code></li>
 | ||
|                 <li><code>kafka.controller:type=ControllerChannelManager,name=RequestRateAndQueueTimeMs</code></li>
 | ||
|                 <li><code>kafka.controller:type=ControllerEventManager,name=EventQueueSize</code></li>
 | ||
|                 <li><code>kafka.controller:type=ControllerEventManager,name=EventQueueTimeMs</code></li>
 | ||
|                 <li><code>kafka.controller:type=ControllerStats,name=AutoLeaderBalanceRateAndTimeMs</code></li>
 | ||
|                 <li><code>kafka.controller:type=ControllerStats,name=ControlledShutdownRateAndTimeMs</code></li>
 | ||
|                 <li><code>kafka.controller:type=ControllerStats,name=ControllerChangeRateAndTimeMs</code></li>
 | ||
|                 <li><code>kafka.controller:type=ControllerStats,name=ControllerShutdownRateAndTimeMs</code></li>
 | ||
|                 <li><code>kafka.controller:type=ControllerStats,name=IdleRateAndTimeMs</code></li>
 | ||
|                 <li><code>kafka.controller:type=ControllerStats,name=IsrChangeRateAndTimeMs</code></li>
 | ||
|                 <li><code>kafka.controller:type=ControllerStats,name=LeaderAndIsrResponseReceivedRateAndTimeMs</code></li>
 | ||
|                 <li><code>kafka.controller:type=ControllerStats,name=LeaderElectionRateAndTimeMs</code></li>
 | ||
|                 <li><code>kafka.controller:type=ControllerStats,name=ListPartitionReassignmentRateAndTimeMs</code></li>
 | ||
|                 <li><code>kafka.controller:type=ControllerStats,name=LogDirChangeRateAndTimeMs</code></li>
 | ||
|                 <li><code>kafka.controller:type=ControllerStats,name=ManualLeaderBalanceRateAndTimeMs</code></li>
 | ||
|                 <li><code>kafka.controller:type=KafkaController,name=MigratingZkBrokerCount</code></li>
 | ||
|                 <li><code>kafka.controller:type=ControllerStats,name=PartitionReassignmentRateAndTimeMs</code></li>
 | ||
|                 <li><code>kafka.controller:type=ControllerStats,name=TopicChangeRateAndTimeMs</code></li>
 | ||
|                 <li><code>kafka.controller:type=ControllerStats,name=TopicDeletionRateAndTimeMs</code></li>
 | ||
|                 <li><code>kafka.controller:type=KafkaController,name=TopicsIneligibleToDeleteCount</code></li>
 | ||
|                 <li><code>kafka.controller:type=ControllerStats,name=TopicUncleanLeaderElectionEnableRateAndTimeMs</code></li>
 | ||
|                 <li><code>kafka.controller:type=ControllerStats,name=UncleanLeaderElectionEnableRateAndTimeMs</code></li>
 | ||
|                 <li><code>kafka.controller:type=ControllerStats,name=UncleanLeaderElectionsPerSec</code></li>
 | ||
|                 <li><code>kafka.controller:type=ControllerStats,name=UpdateFeaturesRateAndTimeMs</code></li>
 | ||
|                 <li><code>kafka.controller:type=ControllerStats,name=UpdateMetadataResponseReceivedRateAndTimeMs</code></li>
 | ||
|                 <li><code>kafka.controller:type=KafkaController,name=ActiveBrokerCount</code></li>
 | ||
|                 <li><code>kafka.controller:type=KafkaController,name=ActiveControllerCount</code></li>
 | ||
|                 <li><code>kafka.controller:type=KafkaController,name=ControllerState</code></li>
 | ||
|                 <li><code>kafka.controller:type=KafkaController,name=FencedBrokerCount</code></li>
 | ||
|                 <li><code>kafka.controller:type=KafkaController,name=GlobalPartitionCount</code></li>
 | ||
|                 <li><code>kafka.controller:type=KafkaController,name=GlobalTopicCount</code></li>
 | ||
|                 <li><code>kafka.controller:type=KafkaController,name=OfflinePartitionsCount</code></li>
 | ||
|                 <li><code>kafka.controller:type=KafkaController,name=PreferredReplicaImbalanceCount</code></li>
 | ||
|                 <li><code>kafka.controller:type=KafkaController,name=ReplicasIneligibleToDeleteCount</code></li>
 | ||
|                 <li><code>kafka.controller:type=KafkaController,name=ReplicasToDeleteCount</code></li>
 | ||
|                 <li><code>kafka.controller:type=KafkaController,name=TopicsToDeleteCount</code></li>
 | ||
|                 <li><code>kafka.controller:type=KafkaController,name=ZkMigrationState</code></li>
 | ||
|                 <li><code>kafka.server:type=DelayedOperationPurgatory,name=PurgatorySize,delayedOperation=ElectLeader</code></li>
 | ||
|                 <li><code>kafka.server:type=DelayedOperationPurgatory,name=PurgatorySize,delayedOperation=topic</code></li>
 | ||
|                 <li><code>kafka.server:type=DelayedOperationPurgatory,name=NumDelayedOperations,delayedOperation=ElectLeader</code></li>
 | ||
|                 <li><code>kafka.server:type=DelayedOperationPurgatory,name=NumDelayedOperations,delayedOperation=topic</code></li>
 | ||
|                 <li><code>kafka.server:type=SessionExpireListener,name=SessionState</code></li>
 | ||
|                 <li><code>kafka.server:type=SessionExpireListener,name=ZooKeeperAuthFailuresPerSec</code></li>
 | ||
|                 <li><code>kafka.server:type=SessionExpireListener,name=ZooKeeperDisconnectsPerSec</code></li>
 | ||
|                 <li><code>kafka.server:type=SessionExpireListener,name=ZooKeeperExpiresPerSec</code></li>
 | ||
|                 <li><code>kafka.server:type=SessionExpireListener,name=ZooKeeperReadOnlyConnectsPerSec</code></li>
 | ||
|                 <li><code>kafka.server:type=SessionExpireListener,name=ZooKeeperSaslAuthenticationsPerSec</code></li>
 | ||
|                 <li><code>kafka.server:type=SessionExpireListener,name=ZooKeeperSyncConnectsPerSec</code></li>
 | ||
|                 <li><code>kafka.server:type=ZooKeeperClientMetrics,name=ZooKeeperRequestLatencyMs</code></li>
 | ||
|             </ul>
 | ||
|         </li>
 | ||
|     </ul>
 | ||
|     <h2 class="anchor-heading">Behavioral Change Reference</h2>
 | ||
|     <p>
 | ||
|         This document catalogs the functional and operational differences between ZooKeeper mode and KRaft mode.
 | ||
|     </p>
 | ||
|     <ul>
 | ||
|         <li>
 | ||
|             <strong>Configuration Value Size Limitation</strong>: KRaft mode restricts configuration values to a maximum size of <code>Short.MAX_VALUE</code>,
 | ||
|             which prevents using the append operation to create larger configuration values.
 | ||
|         </li>
 | ||
|         <li>
 | ||
|             <strong>Policy Class Deployment</strong>:
 | ||
|             In KRaft mode, the <code>CreateTopicPolicy</code> and <code>AlterConfigPolicy</code> plugins run on the controller instead of the broker.
 | ||
|             This requires users to deploy the policy class JAR files on the controller and configure the parameters
 | ||
|             (<code>create.topic.policy.class.name</code> and <code>alter.config.policy.class.name</code>) on the controller.
 | ||
|             <p>Note: If migrating from ZooKeeper mode, ensure policy JARs are moved from brokers to controllers.</p>
 | ||
|         </li>
 | ||
|         <li>
 | ||
|             <strong>Custom implementations of <code>KafkaPrincipalBuilder</code></strong>:
 | ||
|             In KRaft mode, custom implementations of <code>KafkaPrincipalBuilder</code> must also implement <code>KafkaPrincipalSerde</code>; otherwise brokers will not be able to
 | ||
|             forward requests to the controller.
 | ||
|         </li>
 | ||
|     </ul>
 | ||
| </div>
 | ||
| <!--#include virtual="../../includes/_footer.htm" -->
 | ||
| </div>
 |