You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by Anthony Pastor <an...@stickyads.tv> on 2015/02/04 17:19:05 UTC

Topic migration mentionning one removed server

Hello,

I'm trying to understand an awkward issue we're having on our Kafka
production.
We currently have 8 Kafka servers named:
kafka1, kafka2, kafka3 ...

Few weeks ago we migrated all topics present on kafka7 and then have
shutdown it.

The main issue is a performance issue on Kafka5.
Topic migration are very slow.

Today i tried to do a topic migration from kafka2 to kafka5.
The awkward part is that we've kafka7 references like if it's on
replication on this server :

Here are my commands & comments:

- Original state of the topic was :

$ /usr/share/kafka/bin/kafka-topics.sh --zookeeper 172.16.22.1 --describe
--topic AdBidderZoneDealDailyKey
Topic:AdBidderZoneDealDailyKey    PartitionCount:1
ReplicationFactor:2    Configs:retention.ms=604800000
    Topic: AdBidderZoneDealDailyKey    Partition: 0    Leader: 4
Replicas: 4,2    Isr: 4,2


- Then i ran a topic migration :

$ /usr/share/kafka/bin/kafka-reassign-partitions.sh --zookeeper 172.16.22.1
--reassignment-json-file target-zone-deal-daily.json --execute

$ cat target-zone-deal-daily.json
{"version":1,"partitions":[{"topic":"AdBidderZoneDealDailyKey","partition":0,"replicas":[4,5]}]}


- Now i've two issues : the replication is very very slow, and i've kafka7
occurence (but we've shutdown this machine few weeks ago when all topic
were migrated to other servers) :

$ /usr/share/kafka/bin/kafka-topics.sh --zookeeper 172.16.22.1 --describe
--topic AdBidderZoneDealDailyKey
Topic:AdBidderZoneDealDailyKey    PartitionCount:1
ReplicationFactor:4    Configs:retention.ms=604800000
    Topic: AdBidderZoneDealDailyKey    Partition: 0    Leader: 4
Replicas: 4,5,2,*7*    Isr: 4,2


Does anyone have an idea on our issue please ?

Regards.


-- 

Anthony Pastor.

<http://www.stickyads.tv/>

[image:
http://www.stickyads.tv/wp-content/uploads/2013/03/StickyADS.tv-Black-Logo-PNG-300x46.png]
<http://www.stickyads.tv/>
PARIS | *MONTPELLIER* | MADRID | MILAN | HAMBURG | LONDON

Re: Topic migration mentionning one removed server

Posted by Anthony Pastor <an...@stickyads.tv>.
Any help on this subject please ?

2015-02-05 10:45 GMT+01:00 Anthony Pastor <an...@stickyads.tv>:

> We're using Kafka 0.8.1.1 on debian 7.7
>
> - Logs when i migrate a specific topic (~20GB) from kafka5 to kafka2 (No
> problem that way):
>     - controller.log: No logs.
>
> - Logs when i migrate the same specific topic from kafka2 to kafka5 (same
> problems as my original mail):
>     - controller.log:
>
> [2015-02-05 10:40:18,693] DEBUG [PartitionsReassignedListener on 1]:
> Partitions reassigned listener fired for path /admin/reassign_partitions.
> Record partitions to be reassigned
> {"version":1,"partitions":[{"topic":"AdBidderZoneDealDailyKey","partition":0,"replicas":[4,5]}]}
> (kafka.controller.PartitionsReassignedListener)
> [2015-02-05 10:40:18,695] INFO [Controller 1]: Handling reassignment of
> partition [AdBidderZoneDealDailyKey,0] to new replicas 4,5
> (kafka.controller.KafkaController)
> [2015-02-05 10:40:18,703] INFO [Controller 1]: New replicas 4,5 for
> partition [AdBidderZoneDealDailyKey,0] being reassigned not yet caught up
> with the leader (kafka.controller.KafkaController)
> [2015-02-05 10:40:18,710] DEBUG [Controller 1]: Updated path
> /brokers/topics/AdBidderZoneDealDailyKey with
> {"version":1,"partitions":{"0":[4,5,2]}} for replica assignment
> (kafka.controller.KafkaController)
> [2015-02-05 10:40:18,710] INFO [Controller 1]: Updated assigned replicas
> for partition [AdBidderZoneDealDailyKey,0] being reassigned to 4,5,2
> (kafka.controller.KafkaController)
> [2015-02-05 10:40:18,710] DEBUG [Controller 1]: Updating leader epoch for
> partition [AdBidderZoneDealDailyKey,0]. (kafka.controller.KafkaController)
> [2015-02-05 10:40:18,718] INFO [Controller 1]: Updated leader epoch for
> partition [AdBidderZoneDealDailyKey,0] to 37
> (kafka.controller.KafkaController)
> [2015-02-05 10:40:18,719] INFO [Replica state machine on controller 1]:
> Invoking state change to NewReplica for replicas
> [Topic=AdBidderZoneDealDailyKey,Partition=0,Replica=5]
> (kafka.controller.ReplicaStateMachine)
> [2015-02-05 10:40:18,721] INFO [Controller 1]: Waiting for new replicas
> 4,5 for partition [AdBidderZoneDealDailyKey,0] being reassigned to catch up
> with the leader (kafka.controller.KafkaController)
> [2015-02-05 10:40:18,721] INFO [AddPartitionsListener on 1]: Add Partition
> triggered {"version":1,"partitions":{"0":[4,5,2]}} for path
> /brokers/topics/AdBidderZoneDealDailyKey
> (kafka.controller.PartitionStateMachine$AddPartitionsListener)
> [2015-02-05 10:40:18,723] DEBUG [ReassignedPartitionsIsrChangeListener on
> controller 1]: Reassigned partitions isr change listener fired for path
> /brokers/topics/AdBidderZoneDealDailyKey/partitions/0/state with children
> {"controller_epoch":5,"leader":4,"version":1,"leader_epoch":37,"isr":[4,2]}
> (kafka.controller.ReassignedPartitionsIsrChangeListener)
> [2015-02-05 10:40:18,724] INFO [ReassignedPartitionsIsrChangeListener on
> controller 1]: 1/2 replicas have caught up with the leader for partition
> [AdBidderZoneDealDailyKey,0] being reassigned.Replica(s) 5 still need to
> catch up (kafka.controller.ReassignedPartitionsIsrChangeListener)
> [2015-02-05 10:40:18,852] INFO [AddPartitionsListener on 1]: Add Partition
> triggered {"version":1,"partitions":{"0":[4,5,2,7]}} for path
> /brokers/topics/AdBidderZoneDealDailyKey
> (kafka.controller.PartitionStateMachine$AddPartitionsListener)
>
> 2015-02-05 6:29 GMT+01:00 Jun Rao <ju...@confluent.io>:
>
>> Which version of Kafka are you using? Anything interesting from the
>> controller log?
>>
>> Thanks,
>>
>> Jun
>>
>> On Wed, Feb 4, 2015 at 8:19 AM, Anthony Pastor <an...@stickyads.tv>
>> wrote:
>>
>> > Hello,
>> >
>> > I'm trying to understand an awkward issue we're having on our Kafka
>> > production.
>> > We currently have 8 Kafka servers named:
>> > kafka1, kafka2, kafka3 ...
>> >
>> > Few weeks ago we migrated all topics present on kafka7 and then have
>> > shutdown it.
>> >
>> > The main issue is a performance issue on Kafka5.
>> > Topic migration are very slow.
>> >
>> > Today i tried to do a topic migration from kafka2 to kafka5.
>> > The awkward part is that we've kafka7 references like if it's on
>> > replication on this server :
>> >
>> > Here are my commands & comments:
>> >
>> > - Original state of the topic was :
>> >
>> > $ /usr/share/kafka/bin/kafka-topics.sh --zookeeper 172.16.22.1
>> --describe
>> > --topic AdBidderZoneDealDailyKey
>> > Topic:AdBidderZoneDealDailyKey    PartitionCount:1
>> > ReplicationFactor:2    Configs:retention.ms=604800000
>> >     Topic: AdBidderZoneDealDailyKey    Partition: 0    Leader: 4
>> > Replicas: 4,2    Isr: 4,2
>> >
>> >
>> > - Then i ran a topic migration :
>> >
>> > $ /usr/share/kafka/bin/kafka-reassign-partitions.sh --zookeeper
>> > 172.16.22.1 --reassignment-json-file target-zone-deal-daily.json
>> --execute
>> >
>> > $ cat target-zone-deal-daily.json
>> >
>> >
>> {"version":1,"partitions":[{"topic":"AdBidderZoneDealDailyKey","partition":0,"replicas":[4,5]}]}
>> >
>> >
>> > - Now i've two issues : the replication is very very slow, and i've
>> kafka7
>> > occurence (but we've shutdown this machine few weeks ago when all topic
>> > were migrated to other servers) :
>> >
>> > $ /usr/share/kafka/bin/kafka-topics.sh --zookeeper 172.16.22.1
>> --describe
>> > --topic AdBidderZoneDealDailyKey
>> > Topic:AdBidderZoneDealDailyKey    PartitionCount:1
>> > ReplicationFactor:4    Configs:retention.ms=604800000
>> >     Topic: AdBidderZoneDealDailyKey    Partition: 0    Leader: 4
>> > Replicas: 4,5,2,*7*    Isr: 4,2
>> >
>> >
>> > Does anyone have an idea on our issue please ?
>> >
>> > Regards.
>> >
>> >
>> > --
>> >
>> > Anthony Pastor.
>> >
>> > <http://www.stickyads.tv/>
>> >
>> > [image:
>> >
>> http://www.stickyads.tv/wp-content/uploads/2013/03/StickyADS.tv-Black-Logo-PNG-300x46.png
>> ]
>> > <http://www.stickyads.tv/>
>> > PARIS | *MONTPELLIER* | MADRID | MILAN | HAMBURG | LONDON
>> >
>>
>
>
>
> --
>
> Anthony Pastor.
>
> <http://www.stickyads.tv/>
>
> [image:
> http://www.stickyads.tv/wp-content/uploads/2013/03/StickyADS.tv-Black-Logo-PNG-300x46.png]
> <http://www.stickyads.tv/>
> PARIS | *MONTPELLIER* | MADRID | MILAN | HAMBURG | LONDON
>



-- 

Anthony Pastor.

<http://www.stickyads.tv/>

[image:
http://www.stickyads.tv/wp-content/uploads/2013/03/StickyADS.tv-Black-Logo-PNG-300x46.png]
<http://www.stickyads.tv/>
PARIS | *MONTPELLIER* | MADRID | MILAN | HAMBURG | LONDON

Re: Topic migration mentionning one removed server

Posted by Anthony Pastor <an...@stickyads.tv>.
We're using Kafka 0.8.1.1 on debian 7.7

- Logs when i migrate a specific topic (~20GB) from kafka5 to kafka2 (No
problem that way):
    - controller.log: No logs.

- Logs when i migrate the same specific topic from kafka2 to kafka5 (same
problems as my original mail):
    - controller.log:

[2015-02-05 10:40:18,693] DEBUG [PartitionsReassignedListener on 1]:
Partitions reassigned listener fired for path /admin/reassign_partitions.
Record partitions to be reassigned
{"version":1,"partitions":[{"topic":"AdBidderZoneDealDailyKey","partition":0,"replicas":[4,5]}]}
(kafka.controller.PartitionsReassignedListener)
[2015-02-05 10:40:18,695] INFO [Controller 1]: Handling reassignment of
partition [AdBidderZoneDealDailyKey,0] to new replicas 4,5
(kafka.controller.KafkaController)
[2015-02-05 10:40:18,703] INFO [Controller 1]: New replicas 4,5 for
partition [AdBidderZoneDealDailyKey,0] being reassigned not yet caught up
with the leader (kafka.controller.KafkaController)
[2015-02-05 10:40:18,710] DEBUG [Controller 1]: Updated path
/brokers/topics/AdBidderZoneDealDailyKey with
{"version":1,"partitions":{"0":[4,5,2]}} for replica assignment
(kafka.controller.KafkaController)
[2015-02-05 10:40:18,710] INFO [Controller 1]: Updated assigned replicas
for partition [AdBidderZoneDealDailyKey,0] being reassigned to 4,5,2
(kafka.controller.KafkaController)
[2015-02-05 10:40:18,710] DEBUG [Controller 1]: Updating leader epoch for
partition [AdBidderZoneDealDailyKey,0]. (kafka.controller.KafkaController)
[2015-02-05 10:40:18,718] INFO [Controller 1]: Updated leader epoch for
partition [AdBidderZoneDealDailyKey,0] to 37
(kafka.controller.KafkaController)
[2015-02-05 10:40:18,719] INFO [Replica state machine on controller 1]:
Invoking state change to NewReplica for replicas
[Topic=AdBidderZoneDealDailyKey,Partition=0,Replica=5]
(kafka.controller.ReplicaStateMachine)
[2015-02-05 10:40:18,721] INFO [Controller 1]: Waiting for new replicas 4,5
for partition [AdBidderZoneDealDailyKey,0] being reassigned to catch up
with the leader (kafka.controller.KafkaController)
[2015-02-05 10:40:18,721] INFO [AddPartitionsListener on 1]: Add Partition
triggered {"version":1,"partitions":{"0":[4,5,2]}} for path
/brokers/topics/AdBidderZoneDealDailyKey
(kafka.controller.PartitionStateMachine$AddPartitionsListener)
[2015-02-05 10:40:18,723] DEBUG [ReassignedPartitionsIsrChangeListener on
controller 1]: Reassigned partitions isr change listener fired for path
/brokers/topics/AdBidderZoneDealDailyKey/partitions/0/state with children
{"controller_epoch":5,"leader":4,"version":1,"leader_epoch":37,"isr":[4,2]}
(kafka.controller.ReassignedPartitionsIsrChangeListener)
[2015-02-05 10:40:18,724] INFO [ReassignedPartitionsIsrChangeListener on
controller 1]: 1/2 replicas have caught up with the leader for partition
[AdBidderZoneDealDailyKey,0] being reassigned.Replica(s) 5 still need to
catch up (kafka.controller.ReassignedPartitionsIsrChangeListener)
[2015-02-05 10:40:18,852] INFO [AddPartitionsListener on 1]: Add Partition
triggered {"version":1,"partitions":{"0":[4,5,2,7]}} for path
/brokers/topics/AdBidderZoneDealDailyKey
(kafka.controller.PartitionStateMachine$AddPartitionsListener)

2015-02-05 6:29 GMT+01:00 Jun Rao <ju...@confluent.io>:

> Which version of Kafka are you using? Anything interesting from the
> controller log?
>
> Thanks,
>
> Jun
>
> On Wed, Feb 4, 2015 at 8:19 AM, Anthony Pastor <an...@stickyads.tv>
> wrote:
>
> > Hello,
> >
> > I'm trying to understand an awkward issue we're having on our Kafka
> > production.
> > We currently have 8 Kafka servers named:
> > kafka1, kafka2, kafka3 ...
> >
> > Few weeks ago we migrated all topics present on kafka7 and then have
> > shutdown it.
> >
> > The main issue is a performance issue on Kafka5.
> > Topic migration are very slow.
> >
> > Today i tried to do a topic migration from kafka2 to kafka5.
> > The awkward part is that we've kafka7 references like if it's on
> > replication on this server :
> >
> > Here are my commands & comments:
> >
> > - Original state of the topic was :
> >
> > $ /usr/share/kafka/bin/kafka-topics.sh --zookeeper 172.16.22.1 --describe
> > --topic AdBidderZoneDealDailyKey
> > Topic:AdBidderZoneDealDailyKey    PartitionCount:1
> > ReplicationFactor:2    Configs:retention.ms=604800000
> >     Topic: AdBidderZoneDealDailyKey    Partition: 0    Leader: 4
> > Replicas: 4,2    Isr: 4,2
> >
> >
> > - Then i ran a topic migration :
> >
> > $ /usr/share/kafka/bin/kafka-reassign-partitions.sh --zookeeper
> > 172.16.22.1 --reassignment-json-file target-zone-deal-daily.json
> --execute
> >
> > $ cat target-zone-deal-daily.json
> >
> >
> {"version":1,"partitions":[{"topic":"AdBidderZoneDealDailyKey","partition":0,"replicas":[4,5]}]}
> >
> >
> > - Now i've two issues : the replication is very very slow, and i've
> kafka7
> > occurence (but we've shutdown this machine few weeks ago when all topic
> > were migrated to other servers) :
> >
> > $ /usr/share/kafka/bin/kafka-topics.sh --zookeeper 172.16.22.1 --describe
> > --topic AdBidderZoneDealDailyKey
> > Topic:AdBidderZoneDealDailyKey    PartitionCount:1
> > ReplicationFactor:4    Configs:retention.ms=604800000
> >     Topic: AdBidderZoneDealDailyKey    Partition: 0    Leader: 4
> > Replicas: 4,5,2,*7*    Isr: 4,2
> >
> >
> > Does anyone have an idea on our issue please ?
> >
> > Regards.
> >
> >
> > --
> >
> > Anthony Pastor.
> >
> > <http://www.stickyads.tv/>
> >
> > [image:
> >
> http://www.stickyads.tv/wp-content/uploads/2013/03/StickyADS.tv-Black-Logo-PNG-300x46.png
> ]
> > <http://www.stickyads.tv/>
> > PARIS | *MONTPELLIER* | MADRID | MILAN | HAMBURG | LONDON
> >
>



-- 

Anthony Pastor.

<http://www.stickyads.tv/>

[image:
http://www.stickyads.tv/wp-content/uploads/2013/03/StickyADS.tv-Black-Logo-PNG-300x46.png]
<http://www.stickyads.tv/>
PARIS | *MONTPELLIER* | MADRID | MILAN | HAMBURG | LONDON

Re: Topic migration mentionning one removed server

Posted by Jun Rao <ju...@confluent.io>.
Which version of Kafka are you using? Anything interesting from the
controller log?

Thanks,

Jun

On Wed, Feb 4, 2015 at 8:19 AM, Anthony Pastor <an...@stickyads.tv> wrote:

> Hello,
>
> I'm trying to understand an awkward issue we're having on our Kafka
> production.
> We currently have 8 Kafka servers named:
> kafka1, kafka2, kafka3 ...
>
> Few weeks ago we migrated all topics present on kafka7 and then have
> shutdown it.
>
> The main issue is a performance issue on Kafka5.
> Topic migration are very slow.
>
> Today i tried to do a topic migration from kafka2 to kafka5.
> The awkward part is that we've kafka7 references like if it's on
> replication on this server :
>
> Here are my commands & comments:
>
> - Original state of the topic was :
>
> $ /usr/share/kafka/bin/kafka-topics.sh --zookeeper 172.16.22.1 --describe
> --topic AdBidderZoneDealDailyKey
> Topic:AdBidderZoneDealDailyKey    PartitionCount:1
> ReplicationFactor:2    Configs:retention.ms=604800000
>     Topic: AdBidderZoneDealDailyKey    Partition: 0    Leader: 4
> Replicas: 4,2    Isr: 4,2
>
>
> - Then i ran a topic migration :
>
> $ /usr/share/kafka/bin/kafka-reassign-partitions.sh --zookeeper
> 172.16.22.1 --reassignment-json-file target-zone-deal-daily.json --execute
>
> $ cat target-zone-deal-daily.json
>
> {"version":1,"partitions":[{"topic":"AdBidderZoneDealDailyKey","partition":0,"replicas":[4,5]}]}
>
>
> - Now i've two issues : the replication is very very slow, and i've kafka7
> occurence (but we've shutdown this machine few weeks ago when all topic
> were migrated to other servers) :
>
> $ /usr/share/kafka/bin/kafka-topics.sh --zookeeper 172.16.22.1 --describe
> --topic AdBidderZoneDealDailyKey
> Topic:AdBidderZoneDealDailyKey    PartitionCount:1
> ReplicationFactor:4    Configs:retention.ms=604800000
>     Topic: AdBidderZoneDealDailyKey    Partition: 0    Leader: 4
> Replicas: 4,5,2,*7*    Isr: 4,2
>
>
> Does anyone have an idea on our issue please ?
>
> Regards.
>
>
> --
>
> Anthony Pastor.
>
> <http://www.stickyads.tv/>
>
> [image:
> http://www.stickyads.tv/wp-content/uploads/2013/03/StickyADS.tv-Black-Logo-PNG-300x46.png]
> <http://www.stickyads.tv/>
> PARIS | *MONTPELLIER* | MADRID | MILAN | HAMBURG | LONDON
>