[ 
https://issues.apache.org/jira/browse/KAFKA-3390?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Stevo Slavic updated KAFKA-3390:
--------------------------------
    Description: 
For a topic whose deletion has been requested, Kafka replica manager may end up 
infinitely trying and failing to shrink ISR.

Here is fragment from server.log where this recurring and never ending 
condition has been noticed:

{noformat}
[2016-03-04 09:42:13,894] INFO Partition [foo,0] on broker 1: Shrinking ISR for 
partition [foo,0] from 1,3,2 to 1 (kafka.cluster.Partition)
[2016-03-04 09:42:13,897] WARN Conditional update of path 
/brokers/topics/foo/partitions/0/state with data 
{"controller_epoch":53,"leader":1,"version":1,"leader_epoch":34,"isr":[1]} and 
expected version 68 failed due to 
org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode 
for /brokers/topics/foo/partitions/0/state (kafka.utils.ZkUtils)
[2016-03-04 09:42:13,898] INFO Partition [foo,0] on broker 1: Cached zkVersion 
[68] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
[2016-03-04 09:42:23,894] INFO Partition [foo,0] on broker 1: Shrinking ISR for 
partition [foo,0] from 1,3,2 to 1 (kafka.cluster.Partition)
[2016-03-04 09:42:23,897] WARN Conditional update of path 
/brokers/topics/foo/partitions/0/state with data 
{"controller_epoch":53,"leader":1,"version":1,"leader_epoch":34,"isr":[1]} and 
expected version 68 failed due to 
org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode 
for /brokers/topics/foo/partitions/0/state (kafka.utils.ZkUtils)
[2016-03-04 09:42:23,897] INFO Partition [foo,0] on broker 1: Cached zkVersion 
[68] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
[2016-03-04 09:42:33,894] INFO Partition [foo,0] on broker 1: Shrinking ISR for 
partition [foo,0] from 1,3,2 to 1 (kafka.cluster.Partition)
[2016-03-04 09:42:33,897] WARN Conditional update of path 
/brokers/topics/foo/partitions/0/state with data 
{"controller_epoch":53,"leader":1,"version":1,"leader_epoch":34,"isr":[1]} and 
expected version 68 failed due to 
org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode 
for /brokers/topics/foo/partitions/0/state (kafka.utils.ZkUtils)
[2016-03-04 09:42:33,897] INFO Partition [foo,0] on broker 1: Cached zkVersion 
[68] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
...
{noformat}

Before topic deletion was requested, this was state in ZK of its sole partition:
{noformat}
Zxid:           0x1800001045
Cxid:           0xc92
Client id:      0x3532dd88fd20000
Time:           Mon Feb 29 16:46:23 CET 2016
Operation:      setData
Path:           /brokers/topics/foo/partitions/0/state
Data:           
{"controller_epoch":53,"leader":1,"version":1,"leader_epoch":34,"isr":[1,3,2]}
Version:        68
{noformat}

Topic (sole partition) had no data ever published to it. I guess at some point 
after topic deletion has been requested, partition state first got updated and 
this was updated state:
{noformat}
Zxid:           0x180000b0be
Cxid:           0x141e4
Client id:      0x3532dd88fd20000
Time:           Fri Mar 04 9:41:52 CET 2016
Operation:      setData
Path:           /brokers/topics/foo/partitions/0/state
Data:           
{"controller_epoch":54,"leader":1,"version":1,"leader_epoch":35,"isr":[1,3]}
Version:        69
{noformat}

For whatever reason replica manager (some cache it uses, I guess 
ReplicaManager.allPartitions) never sees this update, nor does it see that the 
partition state, partition, partitions node and finally topic node got deleted:
{noformat}
Zxid:           0x180000b0bf
Cxid:           0x40fb
Client id:      0x3532dd88fd2000a
Time:           Fri Mar 04 9:41:52 CET 2016
Operation:      delete
Path:           /brokers/topics/foo/partitions/0/state
---
Zxid:           0x180000b0c0
Cxid:           0x40fe
Client id:      0x3532dd88fd2000a
Time:           Fri Mar 04 9:41:52 CET 2016
Operation:      delete
Path:           /brokers/topics/foo/partitions/0
---
Zxid:           0x180000b0c1
Cxid:           0x4100
Client id:      0x3532dd88fd2000a
Time:           Fri Mar 04 9:41:52 CET 2016
Operation:      delete
Path:           /brokers/topics/foo/partitions
---
Zxid:           0x180000b0c2
Cxid:           0x4102
Client id:      0x3532dd88fd2000a
Time:           Fri Mar 04 9:41:52 CET 2016
Operation:      delete
Path:           /brokers/topics/foo
{noformat}

it just keeps on trying, every {{replica.lag.time.max.ms}}, to shrink ISR even 
for partition/topic that has been deleted.

Broker 1 was controller in the cluster; notice that the same broker was lead 
for the partition before it was deleted.

  was:
For a topic whose deletion has been requested, Kafka replica manager may end up 
infinitely trying and failing to shrink ISR.

Here is fragment from server.log where this recurring and never ending 
condition has been noticed:

{noformat}
[2016-03-04 09:42:13,894] INFO Partition [foo,0] on broker 1: Shrinking ISR for 
partition [foo,0] from 1,3,2 to 1 (kafka.cluster.Partition)
[2016-03-04 09:42:13,897] WARN Conditional update of path 
/brokers/topics/foo/partitions/0/state with data 
{"controller_epoch":53,"leader":1,"version":1,"leader_epoch":34,"isr":[1]} and 
expected version 68 failed due to 
org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode 
for /brokers/topics/foo/partitions/0/state (kafka.utils.ZkUtils)
[2016-03-04 09:42:13,898] INFO Partition [foo,0] on broker 1: Cached zkVersion 
[68] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
[2016-03-04 09:42:23,894] INFO Partition [foo,0] on broker 1: Shrinking ISR for 
partition [foo,0] from 1,3,2 to 1 (kafka.cluster.Partition)
[2016-03-04 09:42:23,897] WARN Conditional update of path 
/brokers/topics/foo/partitions/0/state with data 
{"controller_epoch":53,"leader":1,"version":1,"leader_epoch":34,"isr":[1]} and 
expected version 68 failed due to 
org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode 
for /brokers/topics/foo/partitions/0/state (kafka.utils.ZkUtils)
[2016-03-04 09:42:23,897] INFO Partition [foo,0] on broker 1: Cached zkVersion 
[68] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
[2016-03-04 09:42:33,894] INFO Partition [foo,0] on broker 1: Shrinking ISR for 
partition [foo,0] from 1,3,2 to 1 (kafka.cluster.Partition)
[2016-03-04 09:42:33,897] WARN Conditional update of path 
/brokers/topics/foo/partitions/0/state with data 
{"controller_epoch":53,"leader":1,"version":1,"leader_epoch":34,"isr":[1]} and 
expected version 68 failed due to 
org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode 
for /brokers/topics/foo/partitions/0/state (kafka.utils.ZkUtils)
[2016-03-04 09:42:33,897] INFO Partition [foo,0] on broker 1: Cached zkVersion 
[68] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
...
{noformat}

Before topic deletion was requested, this was state in ZK of its sole partition:
{noformat}
Zxid:           0x1800001045
Cxid:           0xc92
Client id:      0x3532dd88fd20000
Time:           Mon Feb 29 16:46:23 CET 2016
Operation:      setData
Path:           /brokers/topics/foo/partitions/0/state
Data:           
{"controller_epoch":53,"leader":1,"version":1,"leader_epoch":34,"isr":[1,3,2]}
Version:        68
{noformat}

Topic (sole partition) had no data ever published to it. I guess at some point 
after topic deletion has been requested, partition state first got updated and 
this was updated state:
{noformat}
Zxid:           0x180000b0be
Cxid:           0x141e4
Client id:      0x3532dd88fd20000
Time:           Fri Mar 04 9:41:52 CET 2016
Operation:      setData
Path:           /brokers/topics/foo/partitions/0/state
Data:           
{"controller_epoch":54,"leader":1,"version":1,"leader_epoch":35,"isr":[1,3]}
Version:        69
{noformat}

For whatever reason replica manager (some cache it uses, I guess 
ReplicaManager.allPartitions) never sees this update, nor does it see that the 
partition state, partition, partitions node and finally topic node got deleted:
{noformat}
Zxid:           0x180000b0bf
Cxid:           0x40fb
Client id:      0x3532dd88fd2000a
Time:           Fri Mar 04 9:41:52 CET 2016
Operation:      delete
Path:           /brokers/topics/foo/partitions/0/state
---
Zxid:           0x180000b0c0
Cxid:           0x40fe
Client id:      0x3532dd88fd2000a
Time:           Fri Mar 04 9:41:52 CET 2016
Operation:      delete
Path:           /brokers/topics/foo/partitions/0
---
Zxid:           0x180000b0c1
Cxid:           0x4100
Client id:      0x3532dd88fd2000a
Time:           Fri Mar 04 9:41:52 CET 2016
Operation:      delete
Path:           /brokers/topics/foo/partitions
---
Zxid:           0x180000b0c2
Cxid:           0x4102
Client id:      0x3532dd88fd2000a
Time:           Fri Mar 04 9:41:52 CET 2016
Operation:      delete
Path:           /brokers/topics/foo
{noformat}

it just keeps on trying, every {{replica.lag.time.max.ms}}, to shrink ISR even 
for partition/topic that has been deleted.


> ReplicaManager may infinitely try-fail to shrink ISR set of deleted partition
> -----------------------------------------------------------------------------
>
>                 Key: KAFKA-3390
>                 URL: https://issues.apache.org/jira/browse/KAFKA-3390
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 0.9.0.1
>            Reporter: Stevo Slavic
>
> For a topic whose deletion has been requested, Kafka replica manager may end 
> up infinitely trying and failing to shrink ISR.
> Here is fragment from server.log where this recurring and never ending 
> condition has been noticed:
> {noformat}
> [2016-03-04 09:42:13,894] INFO Partition [foo,0] on broker 1: Shrinking ISR 
> for partition [foo,0] from 1,3,2 to 1 (kafka.cluster.Partition)
> [2016-03-04 09:42:13,897] WARN Conditional update of path 
> /brokers/topics/foo/partitions/0/state with data 
> {"controller_epoch":53,"leader":1,"version":1,"leader_epoch":34,"isr":[1]} 
> and expected version 68 failed due to 
> org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = 
> NoNode for /brokers/topics/foo/partitions/0/state (kafka.utils.ZkUtils)
> [2016-03-04 09:42:13,898] INFO Partition [foo,0] on broker 1: Cached 
> zkVersion [68] not equal to that in zookeeper, skip updating ISR 
> (kafka.cluster.Partition)
> [2016-03-04 09:42:23,894] INFO Partition [foo,0] on broker 1: Shrinking ISR 
> for partition [foo,0] from 1,3,2 to 1 (kafka.cluster.Partition)
> [2016-03-04 09:42:23,897] WARN Conditional update of path 
> /brokers/topics/foo/partitions/0/state with data 
> {"controller_epoch":53,"leader":1,"version":1,"leader_epoch":34,"isr":[1]} 
> and expected version 68 failed due to 
> org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = 
> NoNode for /brokers/topics/foo/partitions/0/state (kafka.utils.ZkUtils)
> [2016-03-04 09:42:23,897] INFO Partition [foo,0] on broker 1: Cached 
> zkVersion [68] not equal to that in zookeeper, skip updating ISR 
> (kafka.cluster.Partition)
> [2016-03-04 09:42:33,894] INFO Partition [foo,0] on broker 1: Shrinking ISR 
> for partition [foo,0] from 1,3,2 to 1 (kafka.cluster.Partition)
> [2016-03-04 09:42:33,897] WARN Conditional update of path 
> /brokers/topics/foo/partitions/0/state with data 
> {"controller_epoch":53,"leader":1,"version":1,"leader_epoch":34,"isr":[1]} 
> and expected version 68 failed due to 
> org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = 
> NoNode for /brokers/topics/foo/partitions/0/state (kafka.utils.ZkUtils)
> [2016-03-04 09:42:33,897] INFO Partition [foo,0] on broker 1: Cached 
> zkVersion [68] not equal to that in zookeeper, skip updating ISR 
> (kafka.cluster.Partition)
> ...
> {noformat}
> Before topic deletion was requested, this was state in ZK of its sole 
> partition:
> {noformat}
> Zxid:         0x1800001045
> Cxid:         0xc92
> Client id:    0x3532dd88fd20000
> Time:         Mon Feb 29 16:46:23 CET 2016
> Operation:    setData
> Path:         /brokers/topics/foo/partitions/0/state
> Data:         
> {"controller_epoch":53,"leader":1,"version":1,"leader_epoch":34,"isr":[1,3,2]}
> Version:      68
> {noformat}
> Topic (sole partition) had no data ever published to it. I guess at some 
> point after topic deletion has been requested, partition state first got 
> updated and this was updated state:
> {noformat}
> Zxid:         0x180000b0be
> Cxid:         0x141e4
> Client id:    0x3532dd88fd20000
> Time:         Fri Mar 04 9:41:52 CET 2016
> Operation:    setData
> Path:         /brokers/topics/foo/partitions/0/state
> Data:         
> {"controller_epoch":54,"leader":1,"version":1,"leader_epoch":35,"isr":[1,3]}
> Version:      69
> {noformat}
> For whatever reason replica manager (some cache it uses, I guess 
> ReplicaManager.allPartitions) never sees this update, nor does it see that 
> the partition state, partition, partitions node and finally topic node got 
> deleted:
> {noformat}
> Zxid:         0x180000b0bf
> Cxid:         0x40fb
> Client id:    0x3532dd88fd2000a
> Time:         Fri Mar 04 9:41:52 CET 2016
> Operation:    delete
> Path:         /brokers/topics/foo/partitions/0/state
> ---
> Zxid:         0x180000b0c0
> Cxid:         0x40fe
> Client id:    0x3532dd88fd2000a
> Time:         Fri Mar 04 9:41:52 CET 2016
> Operation:    delete
> Path:         /brokers/topics/foo/partitions/0
> ---
> Zxid:         0x180000b0c1
> Cxid:         0x4100
> Client id:    0x3532dd88fd2000a
> Time:         Fri Mar 04 9:41:52 CET 2016
> Operation:    delete
> Path:         /brokers/topics/foo/partitions
> ---
> Zxid:         0x180000b0c2
> Cxid:         0x4102
> Client id:    0x3532dd88fd2000a
> Time:         Fri Mar 04 9:41:52 CET 2016
> Operation:    delete
> Path:         /brokers/topics/foo
> {noformat}
> it just keeps on trying, every {{replica.lag.time.max.ms}}, to shrink ISR 
> even for partition/topic that has been deleted.
> Broker 1 was controller in the cluster; notice that the same broker was lead 
> for the partition before it was deleted.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to