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

Compare with Current View Page History

« Previous Version 2 Next »

Status

Current stateUnder Discussion

Discussion thread:

JIRA: KAFKA-8305

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

Motivation

In many environments, replication factor can be determined at a cluster level instead of per-topic. This will allow the administrators to set some sane default configuration and make kafka more accessible to engineers in an organization that may not have knowledge to set meaningful values for this configuration.

There is an existing Kafka cluster configuration `default.replication.factor` that currently only applies to topics created automatically (via auto.topic.create), and will be leveraged for this change.

Public Interfaces

  • NewTopic, the Java object passed into AdminClient, will have another constructor that accepts only topic name and number of partitions.
  • A create topic request that specifies the sentinel value of NO_REPLICATION_FACTOR (-1) alongside a valid (non-negative) number of partitions and no manual assignment will create the topic using default.replication.factor. Note that this sentinel value is already sent as part of the request, usually to indicate that manual assignments should be used.

Proposed Changes

The change is straightforward, AdminManager on the broker will modify its verification logic to check for the case described above where partitions are supplied, replication factor is NO_REPLICATION_FACTOR, and manual assignments are not provided. In this case, it will use the default value in the configuration (which defaults to a replication factor of 1).

The request will leverage the existing CreateTopicsRequest/CreateableTopic and the CreateTopicsRespnose. It will also leverage the error code for INVALID_REPLICATION_FACTOR if the value is below 1 (and not -1) or more than the number of brokers.

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.
    • 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 with "replicas must be greater than or equal to zero"). 
  • No labels