Author: Gokul Ramanan Subramanian
Status
Current state: Draft
Discussion thread: here
JIRA:
Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).
Motivation
The number of partitions is a lever in controlling the performance of a Kafka cluster. Increasing the number of partitions can lead to higher availability and performance. However, increasing the number beyond a point can lead to degraded performance on various fronts.
Table 1 shows the impact on producer performance in the context of 3 m5.marge EC2 broker instances, 1 KB-sized records and requested throughput of 240Mbps on a single topic. The table indicates that throughput with 3-way replication improves from 52Mbps to 101Mbps in going from 10 to 100 partitions, but then degrades beyond that. Also, the throughput with 1-way replication is better compared to that with 3-way replication. This is because of the number of replication Fetch requests that a broker receives increases with the number of partitions on the broker, and having 1-way replication means that the broker does not have to deal with Fetch requests. But even so, the performance is much worse with 10000 partitions than with 10 partitions.
Produce throughput | ||||||
---|---|---|---|---|---|---|
Replication | Number of partitions on a broker | |||||
10 | 100 | 500 | 1000 | 5000 | 10000 | |
3-way | 52 | 101 | 86 | 12 | 2.5 | 0.9 |
1-way | 142 | 113 | 24 | 16 |
Table 1: Producer performance test throughput (Mbps) on 3 m5.large brokers, with 1KB-sized records and requested throughput 240 Mbps on a single topic.
Having lots of partitions on a broker can also increase the recovery time in case of broker failure. TODO
We have seen multiple issues in production clusters where having a large number of partitions leads to lower availability, yields live-locked clusters requiring hours of operational effort spent in recovery, and in some cases requires just entirely deleting the clusters and starting all over again implying 100% data loss.
In order to mitigate these issues, we propose having a configuration named max.broker.partitions to limit the number of partitions per broker. Different brokers in a cluster should be configurable with different values for this configuration, in order to support heterogenous clusters where some brokers are more resourceful than others. To support dynamic environments where the resources allocated to a broker may change while the broker is running, it should be possible to modify this configuration dynamically i.e. without restarting the brokers.
Glossary
In this KIP, we will refer to the partition capacity of a broker as the number of partitions it can host beyond what it already hosts. More specifically,
partition capacity of broker = max(0, max.broker.partitions value for broker - current number of partitions hosted by broker)
Public Interfaces
Configs
Config name | Type | Default | Update-mode |
---|---|---|---|
max.broker.partitions | int32 | int32's max value | per-broker |
API Exceptions
CreateTopics, CreatePartitions and AlterPartitionReassignments APIs will throw the following exception if it is not possible to satisfy the request while respecting the max.broker.partitions limit.
public class WillExceedBrokerPartitionCapacityException extends ApiException { ... }
In order to be actionable, the exception message will contain a map of brokers with their respective partition capacities.
Corresponding to this exception, we will have the following API error code.
WILL_EXCEED_BROKER_PARTITION_CAPACITY(88, "Cannot satisfy request without exceeding the max.broker.partitions limit on brokers", WillExceedBrokerPartitionCapacityException::new);
ZooKeeper
Kafka users will be able to use the following to set/modify the max.broker.partitions configuration.
For example,
./kafka-config.sh --zookeeper $ZOOKEEPER --alter --add-config max.broker.partitions=1000 --entity-type brokers --entity-name 1
applies the configuration to broker 1 only, and
./kafka-config.sh --zookeeper $ZOOKEEPER --alter --add-config max.broker.partitions=1000 --entity-type brokers --entity-default
applies the configuration to all brokers.
Proposed Changes
We will introduce a new read-only int32 configuration called max.broker.partitions that Kafka administrators can specify in the server.properties file or modify via ZooKeeper. This will have a default value equal to the maximum int32 value that Java can represent (231 - 1), which should be sufficient for the foreseeable future. In other words, we don't need int64.
With the max.broker.partitions configuration, a topic creation request (via the CreateTopics API) will fail if it is not possible to choose replicas for all the partitions of the topic without running out of partition capacity on at least one broker. This will also hold true for adding partitions to an existing topic (via the CreatePartitions API). Further, a partition reassignment request (via the AlterPartitionReassignments) will fail if the requested reassignment would exceed the max.broker.partitions limit on some broker.
An illustrative example - CreateTopic
For example, in a 3 broker cluster, say that each broker was configured with max.broker.partitions=10. If the brokers already host 8, 6 and 9 partitions respectively, then a request to create a new topic with 1 partition and a replication factor 3 can be satisfied, resulting in partition counts of 9, 7 and 10 respectively. However, the next topic creation request for 1 partition with a replication factor of 3 will fail because broker 3 has already reached the limit. A similar request with a replication factor of 2 can however, succeed, because 2 of the brokers have still not reached the limit. If the original request for a topic with 1 partition was actually a request for a topic with 2 partitions, with a replication factor of 3, then the request would have failed in entirety.
Algorithm for assigning partition replicas to brokers
When creating a topic or adding partitions to a given topic, the controller currently assigns partitions to brokers such that a partition's replicas are on as many different racks as possible (see KIP-36). This algorithm is encapsulated in the method `AdminUtils.assignReplicasToBrokers`. We will modify this algorithm to ensure that brokers with no partition capacity do not get assigned any partitions.
The following sections describe how the controller gets a handle on the max.broker.partitions value and the current number of partitions for all brokers.
Communicating max.broker.partitions value from brokers to controller
TODO
Communicating current number of partitions in broker to controller
Each broker (including the controller broker) keeps a cache of partition state (in `MetadataCache.UpdateMetadataPartitionState`). The `AdminManager` class has access to the `MetadataCache` and will use this to construct a reverse map of `active broker ID → number of partitions`.
Similarly, the `KafkaController` class (which is relevant only on the controller broker) keeps a reference to `ControllerContext`, whose `replicasOnBrokers` method we will use to obtain the number of partitions in each broker.
Compatibility, Deprecation, and Migration Plan
This change is backwards-compatible in practice because we will set the default value for max.broker.partitions equal to the maximum int32 value that Java can represent, which is quite large (231 - 1). Users will anyway run into system issues far before the number of partitions on a broker reaches this value.
In order to ease migration, a broker that already has more than max.broker.partitions number of partitions at the time at which max.broker.partitions configuration is set for that broker, will continue to function just fine for the existing partitions although it will be unable to host any further partitions. The Kafka administrator can later reassign partitions from this broker to another in order to get the broker to respect the max.broker.partitions limit.
Rejected Alternatives
Add configuration to limit number of partitions at the cluster level
Having a limit on each broker effectively imposes a cluster-wide limit as well, thereby making a cluster-level limit not strictly necessary. The cluster-level configuration can make the interface complex, therefore we will avoid it in this KIP. However, we can revisit this in the future if there is a good use case for this.
Add configuration to limit number of partitions at the topic level
Having a limit at the topic level does not help address the problem discussed in this KIP if there are lots of topics. While each topic may only have a limited number of partitions, it is possible that there will be many more partitions than on a broker than it can handle efficiently.
Add configuration to limit the number of topics
Having a limit on the number of topics does not help address the problem discussed in this KIP if there are lots of partitions on the topic.
Make the max.broker.partitions configuration read-only
This approach makes administration a bit easier because once the limit is set on the broker, the users of the cluster cannot accidentally change the value without administrator privileges and without restarting the broker. This can provide an additional safety net.
However, this approach is not flexible. As mentioned in the Motivation section, the number of partitions that a broker can handle depends on the replication factor of these partitions. Smaller the replication factor, lower is the incoming traffic due to replication Fetch requests that the broker has to handle. This allows the broker to use its resources more efficiently to handle the client requests such as produce / consume. Therefore, a Kafka administrator may want to set different values on a broker as the workload changes without disrupting operations by restarting the brokers.
Further, in environments such as Docker where it is possible to allocate more resources to the broker on the fly, it would be restrictive to not be able to modify the max.broker.partitions configuration on the fly as well.