[jira] [Commented] (KAFKA-14190) Corruption of Topic IDs with pre-2.8.0 ZK admin clients

2023-01-17 Thread Justine Olshan (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-14190?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17677877#comment-17677877
 ] 

Justine Olshan commented on KAFKA-14190:


^ +1 to the points above. 
I think that this was covered in point 1 for recovery, but the ZNode can be 
directly written to to give it the correct (original) topic ID. 
This is of course risky as a poorly formatted ZNode will block any topic 
operations.

You can get the old ZNode via: 
{code:java}
get /brokers/topics/{topic}{code}
and copying the entire output `output`.
In the output, the incorrect topic ID should be replaced with the old one (in 
partition.metadata file) to get `modified_output`.

Then the ZNode can be set via :
{code:java}
set /brokers/topics/{topic} modified_output{code}
Then the controller needs to be bounced via deleteall /controller and the 
affected brokers (ones receiving the log messages) also need a restart.


Alternatively, shutting down the broker and deleting the paritition.metadata 
file will recreate the file with the new ID and allow processing to continue.

> Corruption of Topic IDs with pre-2.8.0 ZK admin clients
> ---
>
> Key: KAFKA-14190
> URL: https://issues.apache.org/jira/browse/KAFKA-14190
> Project: Kafka
>  Issue Type: Bug
>  Components: admin, core, zkclient
>Affects Versions: 2.8.0, 3.1.0, 2.8.1, 3.0.0, 3.0.1, 3.2.0, 3.1.1, 3.2.1
>Reporter: Alexandre Dupriez
>Assignee: Divij Vaidya
>Priority: Major
>
> h3. Scope
> The problem reported below has been verified to occur in Zookeeper mode. It 
> has not been attempted with Kraft controllers, although it is unlikely to be 
> reproduced in Kraft mode given the nature of the issue and clients involved.
> h3. Problem Description
> The ID of a topic is lost when an AdminClient of version < 2.8.0 is used to 
> increase the number of partitions of that topic for a cluster with version >= 
> 2.8.0. This results in the controller re-creating the topic IDs upon restart, 
> eventually conflicting with the topic ID of broker’s {{partition.metadata}} 
> files in the partition directories of the impacted topic, leading to an 
> availability loss of the partitions which do not accept leadership / 
> follower-ship when the topic ID indicated by a {{LeaderAndIsr}} request 
> differ from their own locally cached ID.
> One mitigation post-corruption is to substitute the stale topic ID in the 
> {{partition.metadata}} files with the new topic ID referenced by the 
> controller, or alternatively, delete the {{partition.metadata}} file 
> altogether. This requires a restart of the brokers which are assigned the 
> partitions of the impacted topic.
> h3. Steps to reproduce
> 1. Set-up and launch a two-nodes Kafka cluster in Zookeeper mode.
> 2. Create a topic e.g. via {{kafka-topics.sh}}
> {noformat}
> ./bin/kafka-topics.sh --bootstrap-server :9092 --create --topic myTopic 
> --partitions 2 --replication-factor 2{noformat}
> 3. Capture the topic ID using a 2.8.0+ client.
> {noformat}
> ./kafka/bin/kafka-topics.sh --bootstrap-server :9092 --topic myTopic 
> --describe
> Topic: myTopic TopicId: jKTRaM_TSNqocJeQI2aYOQ PartitionCount: 2 
> ReplicationFactor: 2 Configs: segment.bytes=1073741824
> Topic: myTopic Partition: 0 Leader: 0 Replicas: 1,0 Isr: 0,1
> Topic: myTopic Partition: 1 Leader: 0 Replicas: 0,1 Isr: 0,1{noformat}
> 4. Restart one of the broker. This will make each broker create the 
> {{partition.metadata}} files in the partition directories since it will 
> already have loaded the {{Log}} instance in memory.
> 5. Using a *pre-2.8.0* client library, run the following command.
> {noformat}
> ./kafka/bin/kafka-topics.sh --zookeeper :2181 --alter --topic myTopic 
> --partitions 3{noformat}
> 6. Using a 2.8.0+ client library, describe the topic via Zookeeper and notice 
> the absence of topic ID from the output, where it is otherwise expected.
> {noformat}
> ./kafka/bin/kafka-topics.sh —zookeeper :2181 —describe —topic myTopic
> Topic: myTopic PartitionCount: 3 ReplicationFactor: 2 Configs: 
> Topic: myTopic Partition: 0 Leader: 1 Replicas: 1,0 Isr: 0,1
> Topic: myTopic Partition: 1 Leader: 0 Replicas: 0,1 Isr: 0,1
> Topic: myTopic Partition: 2 Leader: 1 Replicas: 1,0 Isr: 1,0{noformat}
> 7. Using a 2.8.0+ client library, describe the topic via a broker endpoint 
> and notice the topic ID changed.
> {noformat}
> ./kafka/bin/kafka-topics.sh —bootstrap-server :9093 —describe —topic myTopic
> Topic: myTopic TopicId: nI-JQtPwQwGiylMfm8k13w PartitionCount: 3 
> ReplicationFactor: 2 Configs: segment.bytes=1073741824
> Topic: myTopic Partition: 0 Leader: 1 Replicas: 1,0 Isr: 1,0
> Topic: myTopic Partition: 1 Leader: 1 Replicas: 0,1 Isr: 1,0
> Topic: myTopic Partition: 2 Leader: 1 Replicas: 1,0 Isr: 1,0{noformat}
> 8. Restart the controller.
> 9. 

[jira] [Commented] (KAFKA-14190) Corruption of Topic IDs with pre-2.8.0 ZK admin clients

2022-12-22 Thread Divij Vaidya (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-14190?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17651225#comment-17651225
 ] 

Divij Vaidya commented on KAFKA-14190:
--

h2. Detection and recovery from this situation

{*}Detection{*}:
 # You can use the script 
[https://gist.github.com/divijvaidya/8a940a3d67bc8b215c95da54f5eff961] to 
detect inconsistency between the topic IDs in Zookeeper vs. the topic IDs that 
are stored in partition.metadata
 # If your cluster is impacted by this bug, you will see the following lines 
state-change.log file on the controller broker 
{noformat}
[2022-08-25 17:44:05,308] ERROR [Broker id=0] Topic Id in memory: 
jKTRaM_TSNqocJeQI2aYOQ does not match the topic Id for partition myTopic-0 
provided in the request: nI-JQtPwQwGiylMfm8k13w. (state.change.logger){noformat}
{*}{*}{*}{*}{*}{*}

{*}Prevention{*}:
 # Use a client >= 2.8.0 OR
 # If you use a client prior to 2.8.0, use it with `--bootstrap-server` instead 
of `–zookeeper`

{*}Recovery{*}:

If you cluster is already impacted by this bug, you can recover using the 
following options:
 # (preferred) Replace the old topic ID in the metadata with the new topic ID. 
You can find the old and the new topic ID in the log line above. 
`jKTRaM_TSNqocJeQI2aYOQ` is the old one and `nI-JQtPwQwGiylMfm8k13w` is the new 
one OR
 # Delete the partition.metadata and restart the brokers which are assigned the 
partitions of the impacted topic.

 

> Corruption of Topic IDs with pre-2.8.0 ZK admin clients
> ---
>
> Key: KAFKA-14190
> URL: https://issues.apache.org/jira/browse/KAFKA-14190
> Project: Kafka
>  Issue Type: Bug
>  Components: admin, core, zkclient
>Affects Versions: 2.8.0, 3.1.0, 2.8.1, 3.0.0, 3.0.1, 3.2.0, 3.1.1, 3.2.1
>Reporter: Alexandre Dupriez
>Priority: Major
>
> h3. Scope
> The problem reported below has been verified to occur in Zookeeper mode. It 
> has not been attempted with Kraft controllers, although it is unlikely to be 
> reproduced in Kraft mode given the nature of the issue and clients involved.
> h3. Problem Description
> The ID of a topic is lost when an AdminClient of version < 2.8.0 is used to 
> increase the number of partitions of that topic for a cluster with version >= 
> 2.8.0. This results in the controller re-creating the topic IDs upon restart, 
> eventually conflicting with the topic ID of broker’s {{partition.metadata}} 
> files in the partition directories of the impacted topic, leading to an 
> availability loss of the partitions which do not accept leadership / 
> follower-ship when the topic ID indicated by a {{LeaderAndIsr}} request 
> differ from their own locally cached ID.
> One mitigation post-corruption is to substitute the stale topic ID in the 
> {{partition.metadata}} files with the new topic ID referenced by the 
> controller, or alternatively, delete the {{partition.metadata}} file 
> altogether. This requires a restart of the brokers which are assigned the 
> partitions of the impacted topic.
> h3. Steps to reproduce
> 1. Set-up and launch a two-nodes Kafka cluster in Zookeeper mode.
> 2. Create a topic e.g. via {{kafka-topics.sh}}
> {noformat}
> ./bin/kafka-topics.sh --bootstrap-server :9092 --create --topic myTopic 
> --partitions 2 --replication-factor 2{noformat}
> 3. Capture the topic ID using a 2.8.0+ client.
> {noformat}
> ./kafka/bin/kafka-topics.sh --bootstrap-server :9092 --topic myTopic 
> --describe
> Topic: myTopic TopicId: jKTRaM_TSNqocJeQI2aYOQ PartitionCount: 2 
> ReplicationFactor: 2 Configs: segment.bytes=1073741824
> Topic: myTopic Partition: 0 Leader: 0 Replicas: 1,0 Isr: 0,1
> Topic: myTopic Partition: 1 Leader: 0 Replicas: 0,1 Isr: 0,1{noformat}
> 4. Restart one of the broker. This will make each broker create the 
> {{partition.metadata}} files in the partition directories since it will 
> already have loaded the {{Log}} instance in memory.
> 5. Using a *pre-2.8.0* client library, run the following command.
> {noformat}
> ./kafka/bin/kafka-topics.sh --zookeeper :2181 --alter --topic myTopic 
> --partitions 3{noformat}
> 6. Using a 2.8.0+ client library, describe the topic via Zookeeper and notice 
> the absence of topic ID from the output, where it is otherwise expected.
> {noformat}
> ./kafka/bin/kafka-topics.sh —zookeeper :2181 —describe —topic myTopic
> Topic: myTopic PartitionCount: 3 ReplicationFactor: 2 Configs: 
> Topic: myTopic Partition: 0 Leader: 1 Replicas: 1,0 Isr: 0,1
> Topic: myTopic Partition: 1 Leader: 0 Replicas: 0,1 Isr: 0,1
> Topic: myTopic Partition: 2 Leader: 1 Replicas: 1,0 Isr: 1,0{noformat}
> 7. Using a 2.8.0+ client library, describe the topic via a broker endpoint 
> and notice the topic ID changed.
> {noformat}
> ./kafka/bin/kafka-topics.sh —bootstrap-server :9093 —describe —topic myTopic
> Topic: myTopic TopicId: 

[jira] [Commented] (KAFKA-14190) Corruption of Topic IDs with pre-2.8.0 ZK admin clients

2022-12-22 Thread Divij Vaidya (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-14190?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17651224#comment-17651224
 ] 

Divij Vaidya commented on KAFKA-14190:
--

Adding reports of users facing this bug which would help us determine priority 
of fixing this.

1. User on mailing list 
[https://lists.apache.org/thread/jzk4tyd1xs1wwj0bpkdnxpw0m152qw1f]

2. User on #kafka channel 
[https://the-asf.slack.com/archives/CE7HWJPHA/p1671529649633529]

 

> Corruption of Topic IDs with pre-2.8.0 ZK admin clients
> ---
>
> Key: KAFKA-14190
> URL: https://issues.apache.org/jira/browse/KAFKA-14190
> Project: Kafka
>  Issue Type: Bug
>  Components: admin, core, zkclient
>Affects Versions: 2.8.0, 3.1.0, 2.8.1, 3.0.0, 3.0.1, 3.2.0, 3.1.1, 3.2.1
>Reporter: Alexandre Dupriez
>Priority: Major
>
> h3. Scope
> The problem reported below has been verified to occur in Zookeeper mode. It 
> has not been attempted with Kraft controllers, although it is unlikely to be 
> reproduced in Kraft mode given the nature of the issue and clients involved.
> h3. Problem Description
> The ID of a topic is lost when an AdminClient of version < 2.8.0 is used to 
> increase the number of partitions of that topic for a cluster with version >= 
> 2.8.0. This results in the controller re-creating the topic IDs upon restart, 
> eventually conflicting with the topic ID of broker’s {{partition.metadata}} 
> files in the partition directories of the impacted topic, leading to an 
> availability loss of the partitions which do not accept leadership / 
> follower-ship when the topic ID indicated by a {{LeaderAndIsr}} request 
> differ from their own locally cached ID.
> One mitigation post-corruption is to substitute the stale topic ID in the 
> {{partition.metadata}} files with the new topic ID referenced by the 
> controller, or alternatively, delete the {{partition.metadata}} file 
> altogether. This requires a restart of the brokers which are assigned the 
> partitions of the impacted topic.
> h3. Steps to reproduce
> 1. Set-up and launch a two-nodes Kafka cluster in Zookeeper mode.
> 2. Create a topic e.g. via {{kafka-topics.sh}}
> {noformat}
> ./bin/kafka-topics.sh --bootstrap-server :9092 --create --topic myTopic 
> --partitions 2 --replication-factor 2{noformat}
> 3. Capture the topic ID using a 2.8.0+ client.
> {noformat}
> ./kafka/bin/kafka-topics.sh --bootstrap-server :9092 --topic myTopic 
> --describe
> Topic: myTopic TopicId: jKTRaM_TSNqocJeQI2aYOQ PartitionCount: 2 
> ReplicationFactor: 2 Configs: segment.bytes=1073741824
> Topic: myTopic Partition: 0 Leader: 0 Replicas: 1,0 Isr: 0,1
> Topic: myTopic Partition: 1 Leader: 0 Replicas: 0,1 Isr: 0,1{noformat}
> 4. Restart one of the broker. This will make each broker create the 
> {{partition.metadata}} files in the partition directories since it will 
> already have loaded the {{Log}} instance in memory.
> 5. Using a *pre-2.8.0* client library, run the following command.
> {noformat}
> ./kafka/bin/kafka-topics.sh --zookeeper :2181 --alter --topic myTopic 
> --partitions 3{noformat}
> 6. Using a 2.8.0+ client library, describe the topic via Zookeeper and notice 
> the absence of topic ID from the output, where it is otherwise expected.
> {noformat}
> ./kafka/bin/kafka-topics.sh —zookeeper :2181 —describe —topic myTopic
> Topic: myTopic PartitionCount: 3 ReplicationFactor: 2 Configs: 
> Topic: myTopic Partition: 0 Leader: 1 Replicas: 1,0 Isr: 0,1
> Topic: myTopic Partition: 1 Leader: 0 Replicas: 0,1 Isr: 0,1
> Topic: myTopic Partition: 2 Leader: 1 Replicas: 1,0 Isr: 1,0{noformat}
> 7. Using a 2.8.0+ client library, describe the topic via a broker endpoint 
> and notice the topic ID changed.
> {noformat}
> ./kafka/bin/kafka-topics.sh —bootstrap-server :9093 —describe —topic myTopic
> Topic: myTopic TopicId: nI-JQtPwQwGiylMfm8k13w PartitionCount: 3 
> ReplicationFactor: 2 Configs: segment.bytes=1073741824
> Topic: myTopic Partition: 0 Leader: 1 Replicas: 1,0 Isr: 1,0
> Topic: myTopic Partition: 1 Leader: 1 Replicas: 0,1 Isr: 1,0
> Topic: myTopic Partition: 2 Leader: 1 Replicas: 1,0 Isr: 1,0{noformat}
> 8. Restart the controller.
> 9. Check the state-change.log file on the controller broker. The following 
> type of logs will appear.
> {noformat}
> [2022-08-25 17:44:05,308] ERROR [Broker id=0] Topic Id in memory: 
> jKTRaM_TSNqocJeQI2aYOQ does not match the topic Id for partition myTopic-0 
> provided in the request: nI-JQtPwQwGiylMfm8k13w. 
> (state.change.logger){noformat}
> 10. Restart the other broker.
> 11. Describe the topic via the broker endpoint or Zookeeper with a 2.8.0+ 
> client library
> {noformat}
> ./kafka/bin/kafka-topics.sh --zookeeper :2181 --describe --topic myTopic
> Topic: myTopic TopicId: nI-JQtPwQwGiylMfm8k13w PartitionCount: 3 
> ReplicationFactor: 2 Configs: 
> 

[jira] [Commented] (KAFKA-14190) Corruption of Topic IDs with pre-2.8.0 ZK admin clients

2022-09-05 Thread Ismael Juma (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-14190?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17600563#comment-17600563
 ] 

Ismael Juma commented on KAFKA-14190:
-

`AdminZkClient` is an internal class and compatibility was never offered (or 
should have been expected) for that. The `–zookeeper` flag for for 
`TopicCommand` has been deprecated since Apache Kafka 2.2.0 
([https://github.com/apache/kafka/blob/2.2.0/core/src/main/scala/kafka/admin/TopicCommand.scala#L492)]
 and removed in 3.0.0.

The issue outlined here is definitely unfortunate and we would probably have 
done something different if we had anticipated it. The best course of action, 
given where we are today, is unclear in my opinion. The simplest path would be 
to discourage usage of any CLI tools older than 3.0.0. They are, by definition, 
unsafe given that they sidestep server validation and security enforcement.

> Corruption of Topic IDs with pre-2.8.0 ZK admin clients
> ---
>
> Key: KAFKA-14190
> URL: https://issues.apache.org/jira/browse/KAFKA-14190
> Project: Kafka
>  Issue Type: Bug
>  Components: admin, core, zkclient
>Affects Versions: 2.8.0, 3.1.0, 2.8.1, 3.0.0, 3.0.1, 3.2.0, 3.1.1, 3.2.1
>Reporter: Alexandre Dupriez
>Priority: Major
>
> h3. Scope
> The problem reported below has been verified to occur in Zookeeper mode. It 
> has not been attempted with Kraft controllers, although it is unlikely to be 
> reproduced in Kraft mode given the nature of the issue and clients involved.
> h3. Problem Description
> The ID of a topic is lost when an AdminClient of version < 2.8.0 is used to 
> increase the number of partitions of that topic for a cluster with version >= 
> 2.8.0. This results in the controller re-creating the topic IDs upon restart, 
> eventually conflicting with the topic ID of broker’s {{partition.metadata}} 
> files in the partition directories of the impacted topic, leading to an 
> availability loss of the partitions which do not accept leadership / 
> follower-ship when the topic ID indicated by a {{LeaderAndIsr}} request 
> differ from their own locally cached ID.
> One mitigation post-corruption is to substitute the stale topic ID in the 
> {{partition.metadata}} files with the new topic ID referenced by the 
> controller, or alternatively, delete the {{partition.metadata}} file 
> altogether. This requires a restart of the brokers which are assigned the 
> partitions of the impacted topic.
> h3. Steps to reproduce
> 1. Set-up and launch a two-nodes Kafka cluster in Zookeeper mode.
> 2. Create a topic e.g. via {{kafka-topics.sh}}
> {noformat}
> ./bin/kafka-topics.sh --bootstrap-server :9092 --create --topic myTopic 
> --partitions 2 --replication-factor 2{noformat}
> 3. Capture the topic ID using a 2.8.0+ client.
> {noformat}
> ./kafka/bin/kafka-topics.sh --bootstrap-server :9092 --topic myTopic 
> --describe
> Topic: myTopic TopicId: jKTRaM_TSNqocJeQI2aYOQ PartitionCount: 2 
> ReplicationFactor: 2 Configs: segment.bytes=1073741824
> Topic: myTopic Partition: 0 Leader: 0 Replicas: 1,0 Isr: 0,1
> Topic: myTopic Partition: 1 Leader: 0 Replicas: 0,1 Isr: 0,1{noformat}
> 4. Restart one of the broker. This will make each broker create the 
> {{partition.metadata}} files in the partition directories since it will 
> already have loaded the {{Log}} instance in memory.
> 5. Using a *pre-2.8.0* client library, run the following command.
> {noformat}
> ./kafka/bin/kafka-topics.sh --zookeeper :2181 --alter --topic myTopic 
> --partitions 3{noformat}
> 6. Using a 2.8.0+ client library, describe the topic via Zookeeper and notice 
> the absence of topic ID from the output, where it is otherwise expected.
> {noformat}
> ./kafka/bin/kafka-topics.sh —zookeeper :2181 —describe —topic myTopic
> Topic: myTopic PartitionCount: 3 ReplicationFactor: 2 Configs: 
> Topic: myTopic Partition: 0 Leader: 1 Replicas: 1,0 Isr: 0,1
> Topic: myTopic Partition: 1 Leader: 0 Replicas: 0,1 Isr: 0,1
> Topic: myTopic Partition: 2 Leader: 1 Replicas: 1,0 Isr: 1,0{noformat}
> 7. Using a 2.8.0+ client library, describe the topic via a broker endpoint 
> and notice the topic ID changed.
> {noformat}
> ./kafka/bin/kafka-topics.sh —bootstrap-server :9093 —describe —topic myTopic
> Topic: myTopic TopicId: nI-JQtPwQwGiylMfm8k13w PartitionCount: 3 
> ReplicationFactor: 2 Configs: segment.bytes=1073741824
> Topic: myTopic Partition: 0 Leader: 1 Replicas: 1,0 Isr: 1,0
> Topic: myTopic Partition: 1 Leader: 1 Replicas: 0,1 Isr: 1,0
> Topic: myTopic Partition: 2 Leader: 1 Replicas: 1,0 Isr: 1,0{noformat}
> 8. Restart the controller.
> 9. Check the state-change.log file on the controller broker. The following 
> type of logs will appear.
> {noformat}
> [2022-08-25 17:44:05,308] ERROR [Broker id=0] Topic Id in memory: 
> jKTRaM_TSNqocJeQI2aYOQ does not match 

[jira] [Commented] (KAFKA-14190) Corruption of Topic IDs with pre-2.8.0 ZK admin clients

2022-08-30 Thread Alexandre Dupriez (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-14190?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17597994#comment-17597994
 ] 

Alexandre Dupriez commented on KAFKA-14190:
---

Hi Ismael,

Thanks for the follow-up. You are right that the problem requires to use the 
{{--zookeeper}} flag (which has been removed from the newest versions). If 
topic changes are applied via broker RPCs, then there is no problem. This 
brings us to your second comment: it is indeed from clients which modify the 
znodes directly. By "admin client", I am specifically referring to the 
{{AdminZkClient}} which accounts for topic IDs in 2.8.0+ versions (and 
[trunk|https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/zk/AdminZkClient.scala#L163-L180])
 but not earlier ones ([e.g. 
2.7.2|https://github.com/apache/kafka/blob/2.7.2/core/src/main/scala/kafka/zk/AdminZkClient.scala#L156-L170]).
 What would be your view on this? Does it qualify as a bug / breaking change?

Thanks,

Alexandre

> Corruption of Topic IDs with pre-2.8.0 ZK admin clients
> ---
>
> Key: KAFKA-14190
> URL: https://issues.apache.org/jira/browse/KAFKA-14190
> Project: Kafka
>  Issue Type: Bug
>  Components: admin, core, zkclient
>Affects Versions: 2.8.0, 3.1.0, 2.8.1, 3.0.0, 3.0.1, 3.2.0, 3.1.1, 3.2.1
>Reporter: Alexandre Dupriez
>Priority: Major
>
> h3. Scope
> The problem reported below has been verified to occur in Zookeeper mode. It 
> has not been attempted with Kraft controllers, although it is unlikely to be 
> reproduced in Kraft mode given the nature of the issue and clients involved.
> h3. Problem Description
> The ID of a topic is lost when an AdminClient of version < 2.8.0 is used to 
> increase the number of partitions of that topic for a cluster with version >= 
> 2.8.0. This results in the controller re-creating the topic IDs upon restart, 
> eventually conflicting with the topic ID of broker’s {{partition.metadata}} 
> files in the partition directories of the impacted topic, leading to an 
> availability loss of the partitions which do not accept leadership / 
> follower-ship when the topic ID indicated by a {{LeaderAndIsr}} request 
> differ from their own locally cached ID.
> One mitigation post-corruption is to substitute the stale topic ID in the 
> {{partition.metadata}} files with the new topic ID referenced by the 
> controller, or alternatively, delete the {{partition.metadata}} file 
> altogether. This requires a restart of the brokers which are assigned the 
> partitions of the impacted topic.
> h3. Steps to reproduce
> 1. Set-up and launch a two-nodes Kafka cluster in Zookeeper mode.
> 2. Create a topic e.g. via {{kafka-topics.sh}}
> {noformat}
> ./bin/kafka-topics.sh --bootstrap-server :9092 --create --topic myTopic 
> --partitions 2 --replication-factor 2{noformat}
> 3. Capture the topic ID using a 2.8.0+ client.
> {noformat}
> ./kafka/bin/kafka-topics.sh --bootstrap-server :9092 --topic myTopic 
> --describe
> Topic: myTopic TopicId: jKTRaM_TSNqocJeQI2aYOQ PartitionCount: 2 
> ReplicationFactor: 2 Configs: segment.bytes=1073741824
> Topic: myTopic Partition: 0 Leader: 0 Replicas: 1,0 Isr: 0,1
> Topic: myTopic Partition: 1 Leader: 0 Replicas: 0,1 Isr: 0,1{noformat}
> 4. Restart one of the broker. This will make each broker create the 
> {{partition.metadata}} files in the partition directories since it will 
> already have loaded the {{Log}} instance in memory.
> 5. Using a *pre-2.8.0* client library, run the following command.
> {noformat}
> ./kafka/bin/kafka-topics.sh --zookeeper :2181 --alter --topic myTopic 
> --partitions 3{noformat}
> 6. Using a 2.8.0+ client library, describe the topic via Zookeeper and notice 
> the absence of topic ID from the output, where it is otherwise expected.
> {noformat}
> ./kafka/bin/kafka-topics.sh —zookeeper :2181 —describe —topic myTopic
> Topic: myTopic PartitionCount: 3 ReplicationFactor: 2 Configs: 
> Topic: myTopic Partition: 0 Leader: 1 Replicas: 1,0 Isr: 0,1
> Topic: myTopic Partition: 1 Leader: 0 Replicas: 0,1 Isr: 0,1
> Topic: myTopic Partition: 2 Leader: 1 Replicas: 1,0 Isr: 1,0{noformat}
> 7. Using a 2.8.0+ client library, describe the topic via a broker endpoint 
> and notice the topic ID changed.
> {noformat}
> ./kafka/bin/kafka-topics.sh —bootstrap-server :9093 —describe —topic myTopic
> Topic: myTopic TopicId: nI-JQtPwQwGiylMfm8k13w PartitionCount: 3 
> ReplicationFactor: 2 Configs: segment.bytes=1073741824
> Topic: myTopic Partition: 0 Leader: 1 Replicas: 1,0 Isr: 1,0
> Topic: myTopic Partition: 1 Leader: 1 Replicas: 0,1 Isr: 1,0
> Topic: myTopic Partition: 2 Leader: 1 Replicas: 1,0 Isr: 1,0{noformat}
> 8. Restart the controller.
> 9. Check the state-change.log file on the controller broker. The following 
> type of logs will appear.
> {noformat}
> 

[jira] [Commented] (KAFKA-14190) Corruption of Topic IDs with pre-2.8.0 ZK admin clients

2022-08-30 Thread Ismael Juma (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-14190?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17597935#comment-17597935
 ] 

Ismael Juma commented on KAFKA-14190:
-

{quote}The ID of a topic is lost when an AdminClient of version < 2.8.0 is used 
to increase the number of partitions of that topic for a cluster with version 
>= 2.8.0
{quote}
Is the above actually true? The command you outlined doesn't use admin client 
at all, it updates zookeeper directly.

> Corruption of Topic IDs with pre-2.8.0 ZK admin clients
> ---
>
> Key: KAFKA-14190
> URL: https://issues.apache.org/jira/browse/KAFKA-14190
> Project: Kafka
>  Issue Type: Bug
>  Components: admin, core, zkclient
>Affects Versions: 2.8.0, 3.1.0, 2.8.1, 3.0.0, 3.0.1, 3.2.0, 3.1.1, 3.2.1
>Reporter: Alexandre Dupriez
>Priority: Major
>
> h3. Scope
> The problem reported below has been verified to occur in Zookeeper mode. It 
> has not been attempted with Kraft controllers, although it is unlikely to be 
> reproduced in Kraft mode given the nature of the issue and clients involved.
> h3. Problem Description
> The ID of a topic is lost when an AdminClient of version < 2.8.0 is used to 
> increase the number of partitions of that topic for a cluster with version >= 
> 2.8.0. This results in the controller re-creating the topic IDs upon restart, 
> eventually conflicting with the topic ID of broker’s {{partition.metadata}} 
> files in the partition directories of the impacted topic, leading to an 
> availability loss of the partitions which do not accept leadership / 
> follower-ship when the topic ID indicated by a {{LeaderAndIsr}} request 
> differ from their own locally cached ID.
> One mitigation post-corruption is to substitute the stale topic ID in the 
> {{partition.metadata}} files with the new topic ID referenced by the 
> controller, or alternatively, delete the {{partition.metadata}} file 
> altogether. This requires a restart of the brokers which are assigned the 
> partitions of the impacted topic.
> h3. Steps to reproduce
> 1. Set-up and launch a two-nodes Kafka cluster in Zookeeper mode.
> 2. Create a topic e.g. via {{kafka-topics.sh}}
> {noformat}
> ./bin/kafka-topics.sh --bootstrap-server :9092 --create --topic myTopic 
> --partitions 2 --replication-factor 2{noformat}
> 3. Capture the topic ID using a 2.8.0+ client.
> {noformat}
> ./kafka/bin/kafka-topics.sh --bootstrap-server :9092 --topic myTopic 
> --describe
> Topic: myTopic TopicId: jKTRaM_TSNqocJeQI2aYOQ PartitionCount: 2 
> ReplicationFactor: 2 Configs: segment.bytes=1073741824
> Topic: myTopic Partition: 0 Leader: 0 Replicas: 1,0 Isr: 0,1
> Topic: myTopic Partition: 1 Leader: 0 Replicas: 0,1 Isr: 0,1{noformat}
> 4. Restart one of the broker. This will make each broker create the 
> {{partition.metadata}} files in the partition directories since it will 
> already have loaded the {{Log}} instance in memory.
> 5. Using a *pre-2.8.0* client library, run the following command.
> {noformat}
> ./kafka/bin/kafka-topics.sh --zookeeper :2181 --alter --topic myTopic 
> --partitions 3{noformat}
> 6. Using a 2.8.0+ client library, describe the topic via Zookeeper and notice 
> the absence of topic ID from the output, where it is otherwise expected.
> {noformat}
> ./kafka/bin/kafka-topics.sh —zookeeper :2181 —describe —topic myTopic
> Topic: myTopic PartitionCount: 3 ReplicationFactor: 2 Configs: 
> Topic: myTopic Partition: 0 Leader: 1 Replicas: 1,0 Isr: 0,1
> Topic: myTopic Partition: 1 Leader: 0 Replicas: 0,1 Isr: 0,1
> Topic: myTopic Partition: 2 Leader: 1 Replicas: 1,0 Isr: 1,0{noformat}
> 7. Using a 2.8.0+ client library, describe the topic via a broker endpoint 
> and notice the topic ID changed.
> {noformat}
> ./kafka/bin/kafka-topics.sh —bootstrap-server :9093 —describe —topic myTopic
> Topic: myTopic TopicId: nI-JQtPwQwGiylMfm8k13w PartitionCount: 3 
> ReplicationFactor: 2 Configs: segment.bytes=1073741824
> Topic: myTopic Partition: 0 Leader: 1 Replicas: 1,0 Isr: 1,0
> Topic: myTopic Partition: 1 Leader: 1 Replicas: 0,1 Isr: 1,0
> Topic: myTopic Partition: 2 Leader: 1 Replicas: 1,0 Isr: 1,0{noformat}
> 8. Restart the controller.
> 9. Check the state-change.log file on the controller broker. The following 
> type of logs will appear.
> {noformat}
> [2022-08-25 17:44:05,308] ERROR [Broker id=0] Topic Id in memory: 
> jKTRaM_TSNqocJeQI2aYOQ does not match the topic Id for partition myTopic-0 
> provided in the request: nI-JQtPwQwGiylMfm8k13w. 
> (state.change.logger){noformat}
> 10. Restart the other broker.
> 11. Describe the topic via the broker endpoint or Zookeeper with a 2.8.0+ 
> client library
> {noformat}
> ./kafka/bin/kafka-topics.sh --zookeeper :2181 --describe --topic myTopic
> Topic: myTopic TopicId: nI-JQtPwQwGiylMfm8k13w PartitionCount: 3 
> ReplicationFactor: 2 

[jira] [Commented] (KAFKA-14190) Corruption of Topic IDs with pre-2.8.0 ZK admin clients

2022-08-30 Thread Ismael Juma (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-14190?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17597928#comment-17597928
 ] 

Ismael Juma commented on KAFKA-14190:
-

Thanks for the JIRA. To clarify, the issue is not using an older admin client, 
it's using an older admin client _and the_ –zookeeper flag, right?

> Corruption of Topic IDs with pre-2.8.0 ZK admin clients
> ---
>
> Key: KAFKA-14190
> URL: https://issues.apache.org/jira/browse/KAFKA-14190
> Project: Kafka
>  Issue Type: Bug
>  Components: admin, core, zkclient
>Affects Versions: 2.8.0, 3.1.0, 2.8.1, 3.0.0, 3.0.1, 3.2.0, 3.1.1, 3.2.1
>Reporter: Alexandre Dupriez
>Priority: Major
>
> h3. Scope
> The problem reported below has been verified to occur in Zookeeper mode. It 
> has not been attempted with Kraft controllers, although it is unlikely to be 
> reproduced in Kraft mode given the nature of the issue and clients involved.
> h3. Problem Description
> The ID of a topic is lost when an AdminClient of version < 2.8.0 is used to 
> increase the number of partitions of that topic for a cluster with version >= 
> 2.8.0. This results in the controller re-creating the topic IDs upon restart, 
> eventually conflicting with the topic ID of broker’s {{partition.metadata}} 
> files in the partition directories of the impacted topic, leading to an 
> availability loss of the partitions which do not accept leadership / 
> follower-ship when the topic ID indicated by a {{LeaderAndIsr}} request 
> differ from their own locally cached ID.
> One mitigation post-corruption is to substitute the stale topic ID in the 
> {{partition.metadata}} files with the new topic ID referenced by the 
> controller, or alternatively, delete the {{partition.metadata}} file 
> altogether. This requires a restart of the brokers which are assigned the 
> partitions of the impacted topic.
> h3. Steps to reproduce
> 1. Set-up and launch a two-nodes Kafka cluster in Zookeeper mode.
> 2. Create a topic e.g. via {{kafka-topics.sh}}
> {noformat}
> ./bin/kafka-topics.sh --bootstrap-server :9092 --create --topic myTopic 
> --partitions 2 --replication-factor 2{noformat}
> 3. Capture the topic ID using a 2.8.0+ client.
> {noformat}
> ./kafka/bin/kafka-topics.sh --bootstrap-server :9092 --topic myTopic 
> --describe
> Topic: myTopic TopicId: jKTRaM_TSNqocJeQI2aYOQ PartitionCount: 2 
> ReplicationFactor: 2 Configs: segment.bytes=1073741824
> Topic: myTopic Partition: 0 Leader: 0 Replicas: 1,0 Isr: 0,1
> Topic: myTopic Partition: 1 Leader: 0 Replicas: 0,1 Isr: 0,1{noformat}
> 4. Restart one of the broker. This will make each broker create the 
> {{partition.metadata}} files in the partition directories since it will 
> already have loaded the {{Log}} instance in memory.
> 5. Using a *pre-2.8.0* client library, run the following command.
> {noformat}
> ./kafka/bin/kafka-topics.sh --zookeeper :2181 --alter --topic myTopic 
> --partitions 3{noformat}
> 6. Using a 2.8.0+ client library, describe the topic via Zookeeper and notice 
> the absence of topic ID from the output, where it is otherwise expected.
> {noformat}
> ./kafka/bin/kafka-topics.sh —zookeeper :2181 —describe —topic myTopic
> Topic: myTopic PartitionCount: 3 ReplicationFactor: 2 Configs: 
> Topic: myTopic Partition: 0 Leader: 1 Replicas: 1,0 Isr: 0,1
> Topic: myTopic Partition: 1 Leader: 0 Replicas: 0,1 Isr: 0,1
> Topic: myTopic Partition: 2 Leader: 1 Replicas: 1,0 Isr: 1,0{noformat}
> 7. Using a 2.8.0+ client library, describe the topic via a broker endpoint 
> and notice the topic ID changed.
> {noformat}
> ./kafka/bin/kafka-topics.sh —bootstrap-server :9093 —describe —topic myTopic
> Topic: myTopic TopicId: nI-JQtPwQwGiylMfm8k13w PartitionCount: 3 
> ReplicationFactor: 2 Configs: segment.bytes=1073741824
> Topic: myTopic Partition: 0 Leader: 1 Replicas: 1,0 Isr: 1,0
> Topic: myTopic Partition: 1 Leader: 1 Replicas: 0,1 Isr: 1,0
> Topic: myTopic Partition: 2 Leader: 1 Replicas: 1,0 Isr: 1,0{noformat}
> 8. Restart the controller.
> 9. Check the state-change.log file on the controller broker. The following 
> type of logs will appear.
> {noformat}
> [2022-08-25 17:44:05,308] ERROR [Broker id=0] Topic Id in memory: 
> jKTRaM_TSNqocJeQI2aYOQ does not match the topic Id for partition myTopic-0 
> provided in the request: nI-JQtPwQwGiylMfm8k13w. 
> (state.change.logger){noformat}
> 10. Restart the other broker.
> 11. Describe the topic via the broker endpoint or Zookeeper with a 2.8.0+ 
> client library
> {noformat}
> ./kafka/bin/kafka-topics.sh --zookeeper :2181 --describe --topic myTopic
> Topic: myTopic TopicId: nI-JQtPwQwGiylMfm8k13w PartitionCount: 3 
> ReplicationFactor: 2 Configs: 
> Topic: myTopic Partition: 0 Leader: 0 Replicas: 1,0 Isr: 0
> Topic: myTopic Partition: 1 Leader: 0 Replicas: 0,1 Isr: 0
> Topic: myTopic