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

Compare with Current View Page History

« Previous Version 15 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 available number of brokers is less than the number of requested replicas, as long as the expected size of the cluster under normal conditions is large enough. As soon as brokers re-join the cluster, the missing replicas will be created. Also a topic should only be created when there are at least enough brokers to satisfy "min.insync.replicas" configuration of the topic.

To allow that, we propose to keep track of brokers that are part of the cluster in Zookeeper even after they disconnect ("observed brokers"), and use that information to determine an assignment (or to determine if a given assignment is feasible) on handling a topic creation request or partition creation request.

When the number of available brokers is less than the replication factor and no assignment is specified, creating an under-replicated topic/partition can be enabled. For example, creating a topic with replication factor 3, in a 3-zones region with 1 broker per region, but a zone is down or a rolling restart is happening.

In case a topic or partition is created under-replicated, the error code will 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.


Zookeeper changes:

New non-ephemeral nodes under /brokers called observed_ids.
Upon starting, brokers will create/update the non ephemeral node for their current id containing the same information as the existing ephemeral brokers/ids node. 

The data in these new nodes (including rack if available) will be used on Topic/Partition creation when the request number of replicas (or the requested broker ids if assignment is specified) are not currently available.

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.

  • Updates to the Decommissioning brokers section in the documentation
    • will mention that if a broker id is never to be reused then its corresponding node in zookeeper /brokers/observed_ids will need to be removed if enable.under.replicated.topic.creation is enabled.

Rejected Alternatives

  • Async replica assignment: Instead of relying on previously known brokers, we considered creating replicas on the available brokers at topic/partition create time and saving in zookeeper some information to asynchronously create missing replicas on the next suitable brokers that would join the cluster. Such a process would involve the controller checking if missing replicas existed when a broker joins the cluster. This alternative felt too complicated. Also the next joining broker may not be on the preferred rack assignment.
  • 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