that's because the topic have replication factor 2 at first, then I want to
reassign all partitions 2 replicas to 1 replicas, so all partitions changed
success except 25 and 31.

I think before the reassign operation partitions 25 and 31 became to have
no leader, cause the reassign operation failed.
2014年7月9日 PM10:33于 "Jun Rao" <jun...@gmail.com>写道:

> It's weird that you have replication factor 1, but two  of the partitions
> 25 and 31 have 2 assigned replicas. What's the command you used for
> reassignment?
>
> Thanks,
>
> Jun
>
>
> On Wed, Jul 9, 2014 at 12:10 AM, 鞠大升 <dashen...@gmail.com> wrote:
>
> > @Jun Rao,   Kafka version: 0.8.1.1
> >
> > @Guozhang Wang, I can not found the original controller log, but I can
> give
> > the controller log after execute ./bin/kafka-reassign-partitions.sh
> > and ./bin/kafka-preferred-replica-election.sh
> >
> > Now I do not known how to recover leader for partition 25 and 31, any
> idea?
> >
> > ----------------- controller log for ./bin/kafka-reassign-partitions.sh
> > -------------------------------------------
> > [2014-07-09 15:01:31,552] DEBUG [PartitionsReassignedListener on 5]:
> > Partitions reassigned listener fired for path /admin/reassign_partitions.
> > Record partitions to be reassigned
> >
> >
> {"version":1,"partitions":[{"topic":"org.mobile_nginx","partition":25,"replicas":[6]},{"topic":"org.mobile_nginx","partition":31,"replicas":[3]}]}
> > (kafka.controller.PartitionsReassignedListener)
> > [2014-07-09 15:01:31,579] INFO [AddPartitionsListener on 5]: Add
> Partition
> > triggered
> >
> >
> {"version":1,"partitions":{"12":[1],"8":[3],"19":[5],"23":[5],"4":[3],"15":[2],"11":[2],"9":[4],"22":[4],"26":[2],"13":[2],"24":[6],"16":[4],"5":[4],"10":[1],"21":[3],"6":[5],"1":[4],"17":[5],"25":[6,1],"14":[4],"31":[3,1],"0":[3],"20":[2],"27":[3],"2":[5],"18":[6],"30":[6],"7":[6],"29":[5],"3":[6],"28":[4]}}
> > for path /brokers/topics/org.mobile_nginx
> > (kafka.controller.PartitionStateMachine$AddPartitionsListener)
> > [2014-07-09 15:01:31,587] DEBUG [PartitionsReassignedListener on 5]:
> > Partitions reassigned listener fired for path /admin/reassign_partitions.
> > Record partitions to be reassigned
> >
> >
> {"version":1,"partitions":[{"topic":"org.mobile_nginx","partition":31,"replicas":[3]}]}
> > (kafka.controller.PartitionsReassignedListener)
> > [2014-07-09 15:01:31,590] INFO [AddPartitionsListener on 5]: Add
> Partition
> > triggered
> >
> >
> {"version":1,"partitions":{"12":[1],"8":[3],"19":[5],"23":[5],"4":[3],"15":[2],"11":[2],"9":[4],"22":[4],"26":[2],"13":[2],"24":[6],"16":[4],"5":[4],"10":[1],"21":[3],"6":[5],"1":[4],"17":[5],"25":[6,1],"14":[4],"31":[3,1],"0":[3],"20":[2],"27":[3],"2":[5],"18":[6],"30":[6],"7":[6],"29":[5],"3":[6],"28":[4]}}
> > for path /brokers/topics/org.mobile_nginx
> > (kafka.controller.PartitionStateMachine$AddPartitionsListener)
> >
> > ----------------- controller log for ./bin/kafka-reassign-partitions.sh
> > -------------------------------------------
> > [2014-07-09 15:07:02,968] DEBUG [PreferredReplicaElectionListener on 5]:
> > Preferred replica election listener fired for path
> > /admin/preferred_replica_election. Record partitions to undergo preferred
> > replica election
> >
> >
> {"version":1,"partitions":[{"topic":"org.mobile_nginx","partition":25},{"topic":"org.mobile_nginx","partition":31}]}
> > (kafka.controller.PreferredReplicaElectionListener)
> > [2014-07-09 15:07:02,969] INFO [Controller 5]: Starting preferred replica
> > leader election for partitions
> [org.mobile_nginx,25],[org.mobile_nginx,31]
> > (kafka.controller.KafkaController)
> > [2014-07-09 15:07:02,969] INFO [Partition state machine on Controller 5]:
> > Invoking state change to OnlinePartition for partitions
> > [org.mobile_nginx,25],[org.mobile_nginx,31]
> > (kafka.controller.PartitionStateMachine)
> > [2014-07-09 15:07:02,972] INFO [PreferredReplicaPartitionLeaderSelector]:
> > Current leader -1 for partition [org.mobile_nginx,25] is not the
> preferred
> > replica. Trigerring preferred replica leader election
> > (kafka.controller.PreferredReplicaPartitionLeaderSelector)
> > [2014-07-09 15:07:02,973] INFO [PreferredReplicaPartitionLeaderSelector]:
> > Current leader -1 for partition [org.mobile_nginx,31] is not the
> preferred
> > replica. Trigerring preferred replica leader election
> > (kafka.controller.PreferredReplicaPartitionLeaderSelector)
> > [2014-07-09 15:07:02,973] WARN [Controller 5]: Partition
> > [org.mobile_nginx,25] failed to complete preferred replica leader
> election.
> > Leader is -1 (kafka.controller.KafkaController)
> > [2014-07-09 15:07:02,973] WARN [Controller 5]: Partition
> > [org.mobile_nginx,31] failed to complete preferred replica leader
> election.
> > Leader is -1 (kafka.controller.KafkaController)
> >
> >
> > On Sun, Jul 6, 2014 at 11:47 PM, Jun Rao <jun...@gmail.com> wrote:
> >
> > > Also, which version of Kafka are you using?
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > >
> > > On Thu, Jul 3, 2014 at 2:26 AM, 鞠大升 <dashen...@gmail.com> wrote:
> > >
> > > > hi, all
> > > >
> > > > I have a topic with 32 partitions, after some reassign operation, 2
> > > > partitions became to no leader and isr.
> > > >
> > > >
> > >
> >
> -------------------------------------------------------------------------------------------------------------------
> > > > Topic:org.mobile_nginx  PartitionCount:32       ReplicationFactor:1
> > > > Configs:
> > > >         Topic: org.mobile_nginx Partition: 0    Leader: 3
> > > Replicas: 3
> > > >     Isr: 3
> > > >         Topic: org.mobile_nginx Partition: 1    Leader: 4
> > > Replicas: 4
> > > >     Isr: 4
> > > >         Topic: org.mobile_nginx Partition: 2    Leader: 5
> > > Replicas: 5
> > > >     Isr: 5
> > > >         Topic: org.mobile_nginx Partition: 3    Leader: 6
> > > Replicas: 6
> > > >     Isr: 6
> > > >         Topic: org.mobile_nginx Partition: 4    Leader: 3
> > > Replicas: 3
> > > >     Isr: 3
> > > >         Topic: org.mobile_nginx Partition: 5    Leader: 4
> > > Replicas: 4
> > > >     Isr: 4
> > > >         Topic: org.mobile_nginx Partition: 6    Leader: 5
> > > Replicas: 5
> > > >     Isr: 5
> > > >         Topic: org.mobile_nginx Partition: 7    Leader: 6
> > > Replicas: 6
> > > >     Isr: 6
> > > >         Topic: org.mobile_nginx Partition: 8    Leader: 3
> > > Replicas: 3
> > > >     Isr: 3
> > > >         Topic: org.mobile_nginx Partition: 9    Leader: 4
> > > Replicas: 4
> > > >     Isr: 4
> > > >         Topic: org.mobile_nginx Partition: 10   Leader: 2
> > > Replicas: 1
> > > >     Isr: 2
> > > >         Topic: org.mobile_nginx Partition: 11   Leader: 2
> > > Replicas: 2
> > > >     Isr: 2
> > > >         Topic: org.mobile_nginx Partition: 12   Leader: 3
> > > Replicas: 1
> > > >     Isr: 3
> > > >         Topic: org.mobile_nginx Partition: 13   Leader: 2
> > > Replicas: 2
> > > >     Isr: 2
> > > >         Topic: org.mobile_nginx Partition: 14   Leader: 4
> > > Replicas: 4
> > > >     Isr: 4
> > > >         Topic: org.mobile_nginx Partition: 15   Leader: 2
> > > Replicas: 2
> > > >     Isr: 2
> > > >         Topic: org.mobile_nginx Partition: 16   Leader: 4
> > > Replicas: 4
> > > >     Isr: 4
> > > >         Topic: org.mobile_nginx Partition: 17   Leader: 5
> > > Replicas: 5
> > > >     Isr: 5
> > > >         Topic: org.mobile_nginx Partition: 18   Leader: 6
> > > Replicas: 6
> > > >     Isr: 6
> > > >         Topic: org.mobile_nginx Partition: 19   Leader: 5
> > > Replicas: 5
> > > >     Isr: 5
> > > >         Topic: org.mobile_nginx Partition: 20   Leader: 2
> > > Replicas: 2
> > > >     Isr: 2
> > > >         Topic: org.mobile_nginx Partition: 21   Leader: 3
> > > Replicas: 3
> > > >     Isr: 3
> > > >         Topic: org.mobile_nginx Partition: 22   Leader: 4
> > > Replicas: 4
> > > >     Isr: 4
> > > >         Topic: org.mobile_nginx Partition: 23   Leader: 5
> > > Replicas: 5
> > > >     Isr: 5
> > > >         Topic: org.mobile_nginx Partition: 24   Leader: 6
> > > Replicas: 6
> > > >     Isr: 6
> > > >         Topic: org.mobile_nginx Partition: 25   Leader: -1
> >  Replicas:
> > > > 6,1   Isr:
> > > >         Topic: org.mobile_nginx Partition: 26   Leader: 2
> > > Replicas: 2
> > > >     Isr: 2
> > > >         Topic: org.mobile_nginx Partition: 27   Leader: 3
> > > Replicas: 3
> > > >     Isr: 3
> > > >         Topic: org.mobile_nginx Partition: 28   Leader: 4
> > > Replicas: 4
> > > >     Isr: 4
> > > >         Topic: org.mobile_nginx Partition: 29   Leader: 5
> > > Replicas: 5
> > > >     Isr: 5
> > > >         Topic: org.mobile_nginx Partition: 30   Leader: 6
> > > Replicas: 6
> > > >     Isr: 6
> > > >         Topic: org.mobile_nginx Partition: 31   Leader: -1
> >  Replicas:
> > > > 3,1   Isr:
> > > >
> > > >
> > >
> >
> -------------------------------------------------------------------------------------------------------------------
> > > > partition-25 and partition-32 have no leader and no isr.
> > > > No matter reassign or leader election operation, can not reduce
> > replicas
> > > > number, and can not election a leader for 4 days.
> > > >
> > > > Anyone have any idea how to resolve this problem?
> > > >
> > > > --
> > > > dashengju
> > > > +86 13810875910
> > > > dashen...@gmail.com
> > > >
> > >
> >
> >
> >
> > --
> > dashengju
> > +86 13810875910
> > dashen...@gmail.com
> >
>

Reply via email to