Thanks Bert for response. Are you saying that if i set replication factor to 3 then two nodes can go down, in that case only 3rd down can serve the data?
In that case(replication factor 3 ) also if some partitions are in synchronization process and meanwhile a node goes down then we may face this issue again. On Tue, Apr 29, 2014 at 2:40 AM, Bert Corderman <bertc...@gmail.com> wrote: > Only a single broker needs to be online for data to be available. In your > example partition 2 and 3 had copies of data on broker 0 and 1. When those > two brokers went down your data was unavailable. To withstand two brokers > going offline you would want to change your replication factor to 3. > > > > One thing to keep in mind…. the replication factor is the TOTAL number of > copies NOT how many follows you have, in other words the replication factor > includes the leader AND the followers. > > > On Mon, Apr 28, 2014 at 11:50 AM, Hanish Bansal < > hanish.bansal.agar...@gmail.com> wrote: > > > After restarting it start working fine. > > > > What i understand from this scenario is: > > > > There must be atleast 2 brokers in isr (In Sync Replica) list of each > topic > > partition before any node goes down which has replica, otherwise topic > may > > go to unstable state. > > Or > > In other words, If topic partitions are in synchronization process and > > meanwhile a node goes down then topic may go to unstable state. > > > > Please let me know if i am missing anything. > > > > > > On Mon, Apr 28, 2014 at 9:19 PM, Hanish Bansal < > > hanish.bansal.agar...@gmail.com> wrote: > > > > > After restarting it start working fine. > > > > > > What i understand from this scenario is: > > > > > > > > > > > > On Mon, Apr 28, 2014 at 8:30 PM, Jun Rao <jun...@gmail.com> wrote: > > > > > >> Partition 2 and 3 are not accessible since both replicas are down. > Have > > >> you > > >> restarted the 2 failed brokers? > > >> > > >> Thanks, > > >> > > >> Jun > > >> > > >> > > >> On Sun, Apr 27, 2014 at 11:58 PM, Hanish Bansal < > > >> hanish.bansal.agar...@gmail.com> wrote: > > >> > > >> > Hi All, > > >> > > > >> > I have 3 nodes kafka cluster with below configuration: > > >> > > > >> > Replication factor: 2 > > >> > Number of partitions: 6 > > >> > > > >> > topic-name: notification > > >> > > > >> > As soon as data is ingested into notification topic, 2 nodes goes > > down. > > >> > After that i am not able to push the data into that topic. > > >> > > > >> > When i check the status using list-topic tool, i am getting below > > >> status: > > >> > > > >> > topic: notification partition: 0 leader: 2 replicas: > 1,2 > > >> > isr: 2 > > >> > topic: notification partition: 1 leader: 2 replicas: > 2,0 > > >> > isr: 2 > > >> > topic: notification partition: 2 leader: -1 replicas: > 0,1 > > >> > isr: > > >> > topic: notification partition: 3 leader: -1 replicas: > 1,0 > > >> > isr: > > >> > topic: notification partition: 4 leader: 2 replicas: > 2,1 > > >> > isr: 2 > > >> > topic: notification partition: 5 leader: 2 replicas: > 0,2 > > >> > isr: 2,0 > > >> > > > >> > Please let me know why i am facing this behaviour ? > > >> > > > >> > -- > > >> > *Thanks & Regards* > > >> > *Hanish Bansal* > > >> > > > >> > > > > > > > > > > > > -- > > > *Thanks & Regards* > > > *Hanish Bansal* > > > > > > > > > > > -- > > *Thanks & Regards* > > *Hanish Bansal* > > > -- *Thanks & Regards* *Hanish Bansal*