[ 
https://issues.apache.org/jira/browse/KAFKA-3959?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15701044#comment-15701044
 ] 

ASF GitHub Bot commented on KAFKA-3959:
---------------------------------------

GitHub user onurkaraman opened a pull request:

    https://github.com/apache/kafka/pull/2177

    KAFKA-3959: enforce offsets.topic.replication.factor

    Kafka brokers have a config called "offsets.topic.replication.factor" that 
specify the replication factor for the "__consumer_offsets" topic. The problem 
is that this config isn't being enforced. If an attempt to create the internal 
topic is made when there are fewer brokers than 
"offsets.topic.replication.factor", the topic ends up getting created anyway 
with the current number of live brokers. The current behavior is pretty 
surprising when you have clients or tooling running as the cluster is getting 
setup. Even if your cluster ends up being huge, you'll find out much later that 
__consumer_offsets was setup with no replication.
    
    The cluster not meeting the "offsets.topic.replication.factor" requirement 
on the internal topic is another way of saying the cluster isn't fully setup 
yet.
    
    The right behavior should be for "offsets.topic.replication.factor" to be 
enforced. Topic creation of the internal topic should fail with 
GROUP_COORDINATOR_NOT_AVAILABLE until the "offsets.topic.replication.factor" 
requirement is met. This closely resembles the behavior of regular topic 
creation when the requested replication factor exceeds the current size of the 
cluster, as the request fails with error INVALID_REPLICATION_FACTOR.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/onurkaraman/kafka KAFKA-3959

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/kafka/pull/2177.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2177
    
----
commit ef8558215ff049f0787ae6df688fd25194b7aae2
Author: Onur Karaman <okara...@linkedin.com>
Date:   2016-11-28T05:29:28Z

    KAFKA-3959: enforce offsets.topic.replication.factor
    
    Kafka brokers have a config called "offsets.topic.replication.factor" that 
specify the replication factor for the "__consumer_offsets" topic. The problem 
is that this config isn't being enforced. If an attempt to create the internal 
topic is made when there are fewer brokers than 
"offsets.topic.replication.factor", the topic ends up getting created anyway 
with the current number of live brokers. The current behavior is pretty 
surprising when you have clients or tooling running as the cluster is getting 
setup. Even if your cluster ends up being huge, you'll find out much later that 
__consumer_offsets was setup with no replication.
    
    The cluster not meeting the "offsets.topic.replication.factor" requirement 
on the internal topic is another way of saying the cluster isn't fully setup 
yet.
    
    The right behavior should be for "offsets.topic.replication.factor" to be 
enforced. Topic creation of the internal topic should fail with 
GROUP_COORDINATOR_NOT_AVAILABLE until the "offsets.topic.replication.factor" 
requirement is met. This closely resembles the behavior of regular topic 
creation when the requested replication factor exceeds the current size of the 
cluster, as the request fails with error INVALID_REPLICATION_FACTOR.

----


> __consumer_offsets wrong number of replicas at startup
> ------------------------------------------------------
>
>                 Key: KAFKA-3959
>                 URL: https://issues.apache.org/jira/browse/KAFKA-3959
>             Project: Kafka
>          Issue Type: Bug
>          Components: consumer, offset manager, replication
>    Affects Versions: 0.9.0.1, 0.10.0.0
>         Environment: Brokers of 3 kafka nodes running Red Hat Enterprise 
> Linux Server release 7.2 (Maipo)
>            Reporter: Alban Hurtaud
>            Assignee: Onur Karaman
>
> When creating a stack of 3 kafka brokers, the consumer is starting faster 
> than kafka nodes and when trying to read a topic, only one kafka node is 
> available.
> So the __consumer_offsets is created with a replication factor set to 1 
> (instead of configured 3) :
> offsets.topic.replication.factor=3
> default.replication.factor=3
> min.insync.replicas=2
> Then, other kafka nodes go up and we have exceptions because the replicas # 
> for __consumer_offsets is 1 and min insync is 2. So exceptions are thrown.
> What I missed is : Why the __consumer_offsets is created with replication to 
> 1 (when 1 broker is running) whereas in server.properties it is set to 3 ?
> To reproduce : 
> - Prepare 3 kafka nodes with the 3 lines above added to servers.properties.
> - Run one kafka,
> - Run one consumer (the __consumer_offsets is created with replicas =1)
> - Run 2 more kafka nodes



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to