You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 6 Next »

Status

Current stateUnder Discussion

Discussion thread: here

JIRA: KAFKA-8305

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

Many simple workloads on Kafka can benefit from default partitioning and replication configurations. This allows system administrators to set some sane defaults, making Kafka more accessible to engineers in an organization that may not have knowledge to set meaningful values for these configurations.

There are two existing Kafka cluster configurations that we well leverage:

  • `num.partitions` - the default number of log partitions per topic
  • `default.replication.factor` - the default replication factor for a topic

Today, these two configurations can only be leveraged from auto.topic.create; this KIP proposes leveraging them from the AdminClient APIs as well.

Public Interfaces

NewTopic will now introduce two new constructors (note that we assume it is unlikely a user will want to specify replicas but not partitions):

public NewTopic(String name); // creates a topic with default partitions and default replication factor
public NewTopic(String name, int numPartitions); // creates a topic with default replication factor

This is what will be passed in AdminClient#createTopics to specify the replication factors. The default values will remain the same (sentinel value of -1).

Proposed Changes

The change is straightforward, AdminManager on the broker will modify its verification logic to the following:

  • (no change) If partitions and replicas are both present (but manual assignment is not), we use the partitions and replicas and determine assignment automatically
  • (no change) If partitions and replicas are both absent and a manual assignment is present, then we use the manual assignment
  • (no change) If either partitions or replicas are present and a manual assignment is present, the request fails
  • (new) if only partitions is present, we will no longer fail and instead use the supplied partitions and the default replication factor
  • (new) if only replicas are present, we will no longer fail and instead use the supplied replication factor and the default partitions
  • (new) if none of partitions, replicas or manual assignment are present, we will no longer fail and instead use the default replication factor and default partitions

The same error codes will be used to convey error: 

    INVALID_PARTITIONS(37, "Number of partitions is below 1.",
            InvalidPartitionsException::new),
    INVALID_REPLICATION_FACTOR(38, "Replication factor is below 1 or larger than the number of available brokers.",
            InvalidReplicationFactorException::new),
    INVALID_REPLICA_ASSIGNMENT(39, "Replica assignment is invalid.",

Compatibility, Deprecation, and Migration Plan

  • No request that was previously valid will become invalid, and no request that was previously valid will change behavior.
  • Some previously invalid requests (e.g. valid partitions and no replication factor or manual assignments) will now succeed.

Rejected Alternatives

  • Add a new configuration to differentiate default.replication.factor for auto-topic-create and for normal behavior (i.e. feature flagging this change). I believe this will just confuse users, and the current configuration name is descriptive enough.
  • Change the broker protocol to accept a new type of request that does not have any value (instead of a sentinel value) for the replication factor/partitions.
    • The original proposal in this KIP will be easier to perform upgrades - since all Kafka brokers will be able to deserialize the incoming request, the only difference is whether or not the request succeeds (i.e. creates the topic) or fails (i.e. sends an error message). 
  • No labels