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

Compare with Current View Page History

« Previous Version 16 Next »

co-authored-by: Mickael Maison <mickael.maison@gmail.com>

co-authored-by: Edoardo Comar <ecomar@uk.ibm.com>

Status

Current state: Draft

Discussion thread: here [Change the link from the KIP proposal email archive to your own email thread]

JIRA: here [Change the link from KAFKA-1 to your own ticket]

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

Motivation

Currently when handling topics or partitions creation requests, Kafka currently enforces all replicas to be created in order to fulfill the request. While all other functionalities (produce/consume) are fault tolerant and can handle some brokers down, topics and partitions creations stop working as soon as there are no enough replicas available. In small clusters, when one node is unavailable, for example when a broker is being restarted, it's possible that there are not enough online brokers to satisfy topic/partition creation even though there are enough brokers to satisfy the "min.insync.replicas" configuration of the topic.

This is obvious in a few scenarios:

  • In a 3 node cluster, while a rolling restart is happening, users can't create topics with replication factor 3. 

  • In a 4 node cluster, while a node is down, a rolling restart also prevent topics with replication factor 3 from being created.

  • in a 9 node cluster with 3 nodes in each zone, if 1 zone was to go offline, the cluster would still contain enough nodes (6) to host a topic with replication factor 3. However, in some environments it may still be preferable to only assign 2 replicas to currently alive nodes (in 2 zones) and assign the last replica to a broker in the unavailable zone that is expected to come back online later.

The same scenarios also apply to adding partitions to existing topics (CreatePartitions API).

Proposed Changes

We would like to allow users to create topics and partitions even when the current available number of brokers is less than the number of requested replicas. It would still require at least enough brokers to satisfy the  "min.insync.replicas" configuration of the topic. Placeholder replica ids (-1, -2, -3) will be inserted for the missing replicas.

When a broker joins the cluster, the controller will check if any partitions have placeholders replicas ids and if so assign this broker as a replica (obviosuly only if this broker is not already a replica).

Because this feature can make partitions appear under replicated, it is disabled by default and can be enabled by a broker configuration: enable.under.replicated.topic.creation.

In case a topic or partition is created under-replicated, the error code will still be NONE in the CreateTopics and CreatePartition response
s.

Public Interfaces

Broker Configuration

NAME

DESCRIPTION

TYPE

DEFAULT

VALID VALUES

IMPORTANCE

DYNAMIC UPDATE MODE

enable.under.replicated.topic.creation

Behavior on Topic or Partition creation without the full set of replicas being active.

BOOLEAN

false

true, false

Medium

cluster-wide

Administrators can disable (false), or enable (true) creations of topics/partitions without the full replication factor. Defaulting to false for compatibility.


Compatibility, Deprecation, and Migration Plan

  • For compatibility the default behavior remains unaltered: a number of active brokers equals (or greater) to the full replication factor is required for successful creation.

  • No migration is necessary.

Rejected Alternatives

  • Store "observed brokers" in Zookeeper and assign replicas to them even if they are offline. Because broker registration is implicit, this alternative required storing extra data in Zookeeper and also required administrator to delete znodes when decommisioning brokers. We cannot tell people to modify ZK directly for this
  • Update CreateTopicsRequest and CreatePartitionsRequest to allow users to disable creation of under replicated topics. This option adds an extra configuration to think abobut to end users and it's unclear how many people would want to take advantageo of such a feature.
  • Update schema of CreateTopicsResponse and CreatePartitionsResponse to contain the actual replication factor at creation. Like assignments, it's not data the client can act on.
  • Create a new error code when a topic/partition is created under replicated. As this is not an error case, it's best to keep returning NONE to avoid breaking existing logic.


  • No labels