kafka-users mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Chris Curtin <curtin.ch...@gmail.com>
Subject Re: Copy availability when broker goes down?
Date Mon, 04 Mar 2013 18:21:41 GMT
I'll grab HEAD in a few minutes and see if the changes.

Issues submitted:

https://issues.apache.org/jira/browse/KAFKA-783

https://issues.apache.org/jira/browse/KAFKA-782

Thanks,

Chris


On Mon, Mar 4, 2013 at 1:15 PM, Jun Rao <junrao@gmail.com> wrote:

> Chris,
>
> As Neha said, the 1st copy of a partition is the preferred replica and we
> try to spread them evenly across the brokers. When a broker is restarted,
> we don't automatically move the leader back to the preferred replica
> though. You will have to run a command line
> tool PreferredReplicaLeaderElectionCommand to balance the leaders again.
>
> Also, I recommend that you try the latest code in 0.8. A bunch of issues
> have been fixes since Jan. You will have to wipe out all your ZK and Kafka
> data first though.
>
> Thanks,
>
> Jun
>
> On Mon, Mar 4, 2013 at 8:32 AM, Chris Curtin <curtin.chris@gmail.com>
> wrote:
>
> > Hi,
> >
> > (Hmm, take 2. Apache's spam filter doesn't like the word to describe the
> > copy of the data. 'R - E -P -L -I -C -A' so it blocked it from sending!
> > Using 'copy' below to mean that concept)
> >
> > I’m running 0.8.0 with HEAD from end of January (not the merge you guys
> did
> > last night).
> >
> > I’m testing how the producer responds to loss of brokers, what errors are
> > produced etc. and noticed some strange things as I shutdown servers in my
> > cluster.
> >
> > Setup:
> > 4 node cluster
> > 1 topic, 3 copies in the set
> > 10 partitions numbered 0-9
> >
> > State of the cluster is determined using TopicMetadataRequest.
> >
> > When I start with a full cluster (2nd column is the partition id, next is
> > leader, then the copy set and ISR):
> >
> > Java: 0:vrd03.atlnp1 R:[  vrd03.atlnp1 vrd04.atlnp1 vrd01.atlnp1] I:[
> > vrd03.atlnp1 vrd04.atlnp1 vrd01.atlnp1]
> > Java: 1:vrd04.atlnp1 R:[  vrd04.atlnp1 vrd01.atlnp1 vrd02.atlnp1] I:[
> > vrd04.atlnp1 vrd01.atlnp1 vrd02.atlnp1]
> > Java: 2:vrd03.atlnp1 R:[  vrd01.atlnp1 vrd02.atlnp1 vrd03.atlnp1] I:[
> > vrd03.atlnp1 vrd01.atlnp1 vrd02.atlnp1]
> > Java: 3:vrd03.atlnp1 R:[  vrd02.atlnp1 vrd03.atlnp1 vrd04.atlnp1] I:[
> > vrd03.atlnp1 vrd04.atlnp1 vrd02.atlnp1]
> > Java: 4:vrd03.atlnp1 R:[  vrd03.atlnp1 vrd01.atlnp1 vrd02.atlnp1] I:[
> > vrd03.atlnp1 vrd01.atlnp1 vrd02.atlnp1]
> > Java: 5:vrd03.atlnp1 R:[  vrd04.atlnp1 vrd02.atlnp1 vrd03.atlnp1] I:[
> > vrd03.atlnp1 vrd04.atlnp1 vrd02.atlnp1]
> > Java: 6:vrd03.atlnp1 R:[  vrd01.atlnp1 vrd03.atlnp1 vrd04.atlnp1] I:[
> > vrd03.atlnp1 vrd04.atlnp1 vrd01.atlnp1]
> > Java: 7:vrd04.atlnp1 R:[  vrd02.atlnp1 vrd04.atlnp1 vrd01.atlnp1] I:[
> > vrd04.atlnp1 vrd01.atlnp1 vrd02.atlnp1]
> > Java: 8:vrd03.atlnp1 R:[  vrd03.atlnp1 vrd02.atlnp1 vrd04.atlnp1] I:[
> > vrd03.atlnp1 vrd04.atlnp1 vrd02.atlnp1]
> > Java: 9:vrd03.atlnp1 R:[  vrd04.atlnp1 vrd03.atlnp1 vrd01.atlnp1] I:[
> > vrd03.atlnp1 vrd04.atlnp1 vrd01.atlnp1]
> >
> > When I stop vrd01, which isn’t leader on any:
> >
> > Java: 0:vrd03.atlnp1 R:[ ] I:[]
> > Java: 1:vrd04.atlnp1 R:[ ] I:[]
> > Java: 2:vrd03.atlnp1 R:[ ] I:[]
> > Java: 3:vrd03.atlnp1 R:[  vrd02.atlnp1 vrd03.atlnp1 vrd04.atlnp1] I:[
> > vrd03.atlnp1 vrd04.atlnp1 vrd02.atlnp1]
> > Java: 4:vrd03.atlnp1 R:[ ] I:[]
> > Java: 5:vrd03.atlnp1 R:[  vrd04.atlnp1 vrd02.atlnp1 vrd03.atlnp1] I:[
> > vrd03.atlnp1 vrd04.atlnp1 vrd02.atlnp1]
> > Java: 6:vrd03.atlnp1 R:[ ] I:[]
> > Java: 7:vrd04.atlnp1 R:[ ] I:[]
> > Java: 8:vrd03.atlnp1 R:[  vrd03.atlnp1 vrd02.atlnp1 vrd04.atlnp1] I:[
> > vrd03.atlnp1 vrd04.atlnp1 vrd02.atlnp1]
> > Java: 9:vrd03.atlnp1 R:[ ] I:[]
> >
> > Does this mean that none of the partitions that used to have a copy on
> > vrd01 are updating ANY of the copies?
> >
> > I ran another test, again starting with a full cluster and all partitions
> > had a full set of copies. When I stop the broker which was leader for 9
> of
> > the 10 partitions, the leaders were all elected on one machine instead of
> > the set of 3. Should the leaders have been better spread out? Also the
> > copies weren’t fully populated either.
> >
> > Last test: started with a full cluster, showing all copies available.
> > Stopped a broker that was not a leader for any partition. Noticed that
> the
> > partitions where the stopped machine was in the copy set didn’t show any
> > copies like above. Let the cluster sit for 30 minutes and didn’t see any
> > new copies being brought on line. How should the cluster handle a machine
> > that is down for an extended period of time?
> >
> > I don’t have a new machine I could add to the cluster, but what happens
> > when I do? Will it not be used until a new topic is added or how does it
> > become a valid option for a copy or eventually the leader?
> >
> > Thanks,
> >
> > Chris
> >
>

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message