> On Mar 7, 2017, at 12:18 PM, James Cheng <wushuja...@gmail.com> wrote:
> 
> 
>> On Mar 7, 2017, at 7:44 AM, Shrikant Patel <spa...@pdxinc.com> wrote:
>> 
>> Thanks for clarification. I am seeing strange behavior in that case,
>> 
>> When I set min.insync.replicas=2 in my server.properties (restart the 
>> server) and set the acks=all on producer, I am still able to publish to 
>> topic even when only leader is up (none of followers are alive). With this 
>> configuration I was hoping to see NotEnoughReplicasException.
>> 
>> When I set min.insync.replicas=2 specifically on my topic and set the 
>> acks=all on producer, I get error NotEnoughReplicasException when less than 
>> 2 replicas are live.
>> 
>> Because of this behavior I thought min.insync.replicas=2 in my 
>> server.properties does not work and we needed it at explicit topic level.
>> 
>> This looks like bug?? Anyone facing this issue.
>> 
> 
> Did you try min.insync.replicas=2 in server.properties and acks=all on 
> producer? min.insync.replicas only applies when acks=all.
> 

Oops, sorry, I didn't read properly. I see that you already tried to do that.

When the broker starts up, it prints out a message to the logs showing all the 
configuration settings. I often use that to double check that I specified my 
settings properly in server.properties.

-James

> -James
> 
>> 
>> -----Original Message-----
>> From: Todd Palino [mailto:tpal...@gmail.com]
>> Sent: Monday, March 06, 2017 6:48 PM
>> To: users@kafka.apache.org
>> Subject: Re: Clarification on min.insync.replicas​
>> 
>> Default broker configurations do not show in the topic overrides (which is 
>> what you are showing with the topics tool). It is more accurate to say that 
>> the min.insync.replicas setting in your server.properties file is what will 
>> apply to every topic (regardless of when it is created), if there exists no 
>> topic override for that configuration for that config.
>> 
>> -Todd
>> 
>> 
>> On Mon, Mar 6, 2017 at 4:38 PM, Shrikant Patel <spa...@pdxinc.com> wrote:
>> 
>>> Hi All,
>>> 
>>> Need details about min.insync.replicas​ in the server.properties.
>>> 
>>> I thought once I add this to server.properties, all subsequent topic
>>> create should have this as default value.
>>> 
>>> C:\JAVA_INSTALLATION\kafka\kafka_2.11-0.10.1.1>bin\windows\kafka-topic
>>> s.bat --zookeeper localhost:2181/chroot/cluster1 --create --topic test
>>> --partition 3 --replication-factor 3 Created topic "test".
>>> 
>>> C:\JAVA_INSTALLATION\kafka\kafka_2.11-0.10.1.1>bin\windows\kafka-topic
>>> s.bat --zookeeper localhost:2181/chroot/cluster1 --describe --topic
>>> test
>>> Topic:test      PartitionCount:3        ReplicationFactor:3     Configs:
>>> 
>>> No min.insync.replicas is set on the topic.
>>> 
>>> Why do I have explicit provide this configuratoin when creating topic?
>>> So whats the purpose of this in server.properties??
>>> 
>>> C:\JAVA_INSTALLATION\kafka\kafka_2.11-0.10.1.1>bin\windows\kafka-topic
>>> s.bat --zookeeper localhost:2181/chroot/cluster1 --create --topic test
>>> --partition 3 --replication-factor 3 --config min.insync.replicas=3
>>> Created topic "test".
>>> 
>>> C:\JAVA_INSTALLATION\kafka\kafka_2.11-0.10.1.1>bin\windows\kafka-topic
>>> s.bat --zookeeper localhost:2181/chroot/cluster1 --describe --topic
>>> test
>>> Topic:test      PartitionCount:3        ReplicationFactor:3
>>> Configs:min.insync.replicas=3
>>>       Topic: test     Partition: 0    Leader: 1       Replicas: 1,2,0
>>> Isr: 1,2,0
>>>       Topic: test     Partition: 1    Leader: 2       Replicas: 2,0,1
>>> Isr: 2,0,1
>>>       Topic: test     Partition: 2    Leader: 0       Replicas: 0,1,2
>>> Isr: 0,1,2
>>> 
>>> Thanks
>>> Shri
>>> 
>>> 
>>> 
>>> This e-mail and its contents (to include attachments) are the property
>>> of National Health Systems, Inc., its subsidiaries and affiliates,
>>> including but not limited to Rx.com Community Healthcare Network, Inc.
>>> and its subsidiaries, and may contain confidential and proprietary or
>>> privileged information. If you are not the intended recipient of this
>>> e-mail, you are hereby notified that any unauthorized disclosure,
>>> copying, or distribution of this e-mail or of its attachments, or the
>>> taking of any unauthorized action based on information contained herein is 
>>> strictly prohibited.
>>> Unauthorized use of information contained herein may subject you to
>>> civil and criminal prosecution and penalties. If you are not the
>>> intended recipient, please immediately notify the sender by telephone
>>> at
>>> 800-433-5719 or return e-mail and permanently delete the original e-mail.
>>> 
>> 
>> 
>> 
>> --
>> *Todd Palino*
>> Staff Site Reliability Engineer
>> Data Infrastructure Streaming
>> 
>> 
>> 
>> linkedin.com/in/toddpalino
>> This e-mail and its contents (to include attachments) are the property of 
>> National Health Systems, Inc., its subsidiaries and affiliates, including 
>> but not limited to Rx.com Community Healthcare Network, Inc. and its 
>> subsidiaries, and may contain confidential and proprietary or privileged 
>> information. If you are not the intended recipient of this e-mail, you are 
>> hereby notified that any unauthorized disclosure, copying, or distribution 
>> of this e-mail or of its attachments, or the taking of any unauthorized 
>> action based on information contained herein is strictly prohibited. 
>> Unauthorized use of information contained herein may subject you to civil 
>> and criminal prosecution and penalties. If you are not the intended 
>> recipient, please immediately notify the sender by telephone at 800-433-5719 
>> or return e-mail and permanently delete the original e-mail.
> 

Reply via email to