[jira] [Commented] (KAFKA-16162) New created topics are unavailable after upgrading to 3.7

2024-02-09 Thread Omnia Ibrahim (Jira)


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

Omnia Ibrahim commented on KAFKA-16162:
---

Marking this as resolved as the pr was committed

> New created topics are unavailable after upgrading to 3.7
> -
>
> Key: KAFKA-16162
> URL: https://issues.apache.org/jira/browse/KAFKA-16162
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 3.7.0
>Reporter: Luke Chen
>Assignee: Gaurav Narula
>Priority: Blocker
> Fix For: 3.7.0
>
>
> In 3.7, we introduced the KIP-858 JBOD feature, and the brokerRegistration 
> request will include the `LogDirs` fields with UUID for each log dir in each 
> broker. This info will be stored in the controller and used to identify if 
> the log dir is known and online while handling AssignReplicasToDirsRequest 
> [here|https://github.com/apache/kafka/blob/trunk/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java#L2093].
>  
> While upgrading from old version, the kafka cluster will run in 3.7 binary 
> with old metadata version, and then upgrade to newer version using 
> kafka-features.sh. That means, while brokers startup and send the 
> brokerRegistration request, it'll be using older metadata version without 
> `LogDirs` fields included. And it makes the controller has no log dir info 
> for all brokers. Later, after upgraded, if new topic is created, the flow 
> will go like this:
> 1. Controller assign replicas and adds in metadata log
> 2. brokers fetch the metadata and apply it
> 3. ReplicaManager#maybeUpdateTopicAssignment will update topic assignment
> 4. After sending ASSIGN_REPLICAS_TO_DIRS to controller with replica 
> assignment, controller will think the log dir in current replica is offline, 
> so triggering offline handler, and reassign leader to another replica, and 
> offline, until no more replicas to assign, so assigning leader to -1 (i.e. no 
> leader) 
> So, the results will be that new created topics are unavailable (with no 
> leader) because the controller thinks all log dir are offline.
> {code:java}
> lukchen@lukchen-mac kafka % bin/kafka-topics.sh --describe --topic 
> quickstart-events3 --bootstrap-server localhost:9092  
> 
> Topic: quickstart-events3 TopicId: s8s6tEQyRvmjKI6ctNTgPg PartitionCount: 
> 3   ReplicationFactor: 3Configs: segment.bytes=1073741824
>   Topic: quickstart-events3   Partition: 0Leader: none
> Replicas: 7,2,6 Isr: 6
>   Topic: quickstart-events3   Partition: 1Leader: none
> Replicas: 2,6,7 Isr: 6
>   Topic: quickstart-events3   Partition: 2Leader: none
> Replicas: 6,7,2 Isr: 6
> {code}
> The log snippet in the controller :
> {code:java}
> # handling 1st assignReplicaToDirs request
> [2024-01-18 19:34:47,370] DEBUG [QuorumController id=1] Broker 6 assigned 
> partition quickstart-events3:0 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,370] DEBUG [QuorumController id=1] Broker 6 assigned 
> partition quickstart-events3:2 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,371] DEBUG [QuorumController id=1] Broker 6 assigned 
> partition quickstart-events3:1 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] 
> offline-dir-assignment: changing partition(s): quickstart-events3-0, 
> quickstart-events3-2, quickstart-events3-1 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] partition change for 
> quickstart-events3-0 with topic ID 6ZIeidfiSTWRiOAmGEwn_g: directories: 
> [AA, AA, AA] -> 
> [7K5JBERyyqFFxIXSXYluJA, AA, AA], 
> partitionEpoch: 0 -> 1 (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] Replayed partition 
> change PartitionChangeRecord(partitionId=0, topicId=6ZIeidfiSTWRiOAmGEwn_g, 
> isr=null, leader=-2, replicas=null, removingReplicas=null, 
> addingReplicas=null, leaderRecoveryState=-1, 
> directories=[7K5JBERyyqFFxIXSXYluJA, AA, 
> AA], eligibleLeaderReplicas=null, lastKnownELR=null) for 
> topic quickstart-events3 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] partition change for 
> quickstart-events3-2 with topic ID 6ZIeidfiSTWRiOAmGEwn_g: directories: 
> [AA, AA, 

[jira] [Commented] (KAFKA-16162) New created topics are unavailable after upgrading to 3.7

2024-01-28 Thread Proven Provenzano (Jira)


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

Proven Provenzano commented on KAFKA-16162:
---

Hey folks!

Sorry, I was unavailable all last week. 

I am comfortable with resending the BrokerRegistration. I think we should 
probably always do it after any MV upgrade as there could be other future 
issues that are similar. The BrokerRegistration is a nice way of updating once 
only state of a Broker.

In a separate issue I think we need to separate Heartbeat updates from 
BrokerRegistration and only use BrokerRegistration to indicate the set of 
possible directories not the set of active directories. A separate record 
should be used to indicate the offline directories of a broker. This isn't 
necessary now and I'll create a Jira for it for future enhancements.

 

> New created topics are unavailable after upgrading to 3.7
> -
>
> Key: KAFKA-16162
> URL: https://issues.apache.org/jira/browse/KAFKA-16162
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 3.7.0
>Reporter: Luke Chen
>Assignee: Gaurav Narula
>Priority: Blocker
>
> In 3.7, we introduced the KIP-858 JBOD feature, and the brokerRegistration 
> request will include the `LogDirs` fields with UUID for each log dir in each 
> broker. This info will be stored in the controller and used to identify if 
> the log dir is known and online while handling AssignReplicasToDirsRequest 
> [here|https://github.com/apache/kafka/blob/trunk/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java#L2093].
>  
> While upgrading from old version, the kafka cluster will run in 3.7 binary 
> with old metadata version, and then upgrade to newer version using 
> kafka-features.sh. That means, while brokers startup and send the 
> brokerRegistration request, it'll be using older metadata version without 
> `LogDirs` fields included. And it makes the controller has no log dir info 
> for all brokers. Later, after upgraded, if new topic is created, the flow 
> will go like this:
> 1. Controller assign replicas and adds in metadata log
> 2. brokers fetch the metadata and apply it
> 3. ReplicaManager#maybeUpdateTopicAssignment will update topic assignment
> 4. After sending ASSIGN_REPLICAS_TO_DIRS to controller with replica 
> assignment, controller will think the log dir in current replica is offline, 
> so triggering offline handler, and reassign leader to another replica, and 
> offline, until no more replicas to assign, so assigning leader to -1 (i.e. no 
> leader) 
> So, the results will be that new created topics are unavailable (with no 
> leader) because the controller thinks all log dir are offline.
> {code:java}
> lukchen@lukchen-mac kafka % bin/kafka-topics.sh --describe --topic 
> quickstart-events3 --bootstrap-server localhost:9092  
> 
> Topic: quickstart-events3 TopicId: s8s6tEQyRvmjKI6ctNTgPg PartitionCount: 
> 3   ReplicationFactor: 3Configs: segment.bytes=1073741824
>   Topic: quickstart-events3   Partition: 0Leader: none
> Replicas: 7,2,6 Isr: 6
>   Topic: quickstart-events3   Partition: 1Leader: none
> Replicas: 2,6,7 Isr: 6
>   Topic: quickstart-events3   Partition: 2Leader: none
> Replicas: 6,7,2 Isr: 6
> {code}
> The log snippet in the controller :
> {code:java}
> # handling 1st assignReplicaToDirs request
> [2024-01-18 19:34:47,370] DEBUG [QuorumController id=1] Broker 6 assigned 
> partition quickstart-events3:0 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,370] DEBUG [QuorumController id=1] Broker 6 assigned 
> partition quickstart-events3:2 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,371] DEBUG [QuorumController id=1] Broker 6 assigned 
> partition quickstart-events3:1 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] 
> offline-dir-assignment: changing partition(s): quickstart-events3-0, 
> quickstart-events3-2, quickstart-events3-1 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] partition change for 
> quickstart-events3-0 with topic ID 6ZIeidfiSTWRiOAmGEwn_g: directories: 
> [AA, AA, AA] -> 
> [7K5JBERyyqFFxIXSXYluJA, AA, AA], 
> partitionEpoch: 0 -> 1 (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] Replayed partition 
> change 

[jira] [Commented] (KAFKA-16162) New created topics are unavailable after upgrading to 3.7

2024-01-26 Thread Gaurav Narula (Jira)


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

Gaurav Narula commented on KAFKA-16162:
---

Raised https://github.com/apache/kafka/pull/15270

> New created topics are unavailable after upgrading to 3.7
> -
>
> Key: KAFKA-16162
> URL: https://issues.apache.org/jira/browse/KAFKA-16162
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 3.7.0
>Reporter: Luke Chen
>Priority: Blocker
>
> In 3.7, we introduced the KIP-858 JBOD feature, and the brokerRegistration 
> request will include the `LogDirs` fields with UUID for each log dir in each 
> broker. This info will be stored in the controller and used to identify if 
> the log dir is known and online while handling AssignReplicasToDirsRequest 
> [here|https://github.com/apache/kafka/blob/trunk/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java#L2093].
>  
> While upgrading from old version, the kafka cluster will run in 3.7 binary 
> with old metadata version, and then upgrade to newer version using 
> kafka-features.sh. That means, while brokers startup and send the 
> brokerRegistration request, it'll be using older metadata version without 
> `LogDirs` fields included. And it makes the controller has no log dir info 
> for all brokers. Later, after upgraded, if new topic is created, the flow 
> will go like this:
> 1. Controller assign replicas and adds in metadata log
> 2. brokers fetch the metadata and apply it
> 3. ReplicaManager#maybeUpdateTopicAssignment will update topic assignment
> 4. After sending ASSIGN_REPLICAS_TO_DIRS to controller with replica 
> assignment, controller will think the log dir in current replica is offline, 
> so triggering offline handler, and reassign leader to another replica, and 
> offline, until no more replicas to assign, so assigning leader to -1 (i.e. no 
> leader) 
> So, the results will be that new created topics are unavailable (with no 
> leader) because the controller thinks all log dir are offline.
> {code:java}
> lukchen@lukchen-mac kafka % bin/kafka-topics.sh --describe --topic 
> quickstart-events3 --bootstrap-server localhost:9092  
> 
> Topic: quickstart-events3 TopicId: s8s6tEQyRvmjKI6ctNTgPg PartitionCount: 
> 3   ReplicationFactor: 3Configs: segment.bytes=1073741824
>   Topic: quickstart-events3   Partition: 0Leader: none
> Replicas: 7,2,6 Isr: 6
>   Topic: quickstart-events3   Partition: 1Leader: none
> Replicas: 2,6,7 Isr: 6
>   Topic: quickstart-events3   Partition: 2Leader: none
> Replicas: 6,7,2 Isr: 6
> {code}
> The log snippet in the controller :
> {code:java}
> # handling 1st assignReplicaToDirs request
> [2024-01-18 19:34:47,370] DEBUG [QuorumController id=1] Broker 6 assigned 
> partition quickstart-events3:0 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,370] DEBUG [QuorumController id=1] Broker 6 assigned 
> partition quickstart-events3:2 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,371] DEBUG [QuorumController id=1] Broker 6 assigned 
> partition quickstart-events3:1 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] 
> offline-dir-assignment: changing partition(s): quickstart-events3-0, 
> quickstart-events3-2, quickstart-events3-1 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] partition change for 
> quickstart-events3-0 with topic ID 6ZIeidfiSTWRiOAmGEwn_g: directories: 
> [AA, AA, AA] -> 
> [7K5JBERyyqFFxIXSXYluJA, AA, AA], 
> partitionEpoch: 0 -> 1 (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] Replayed partition 
> change PartitionChangeRecord(partitionId=0, topicId=6ZIeidfiSTWRiOAmGEwn_g, 
> isr=null, leader=-2, replicas=null, removingReplicas=null, 
> addingReplicas=null, leaderRecoveryState=-1, 
> directories=[7K5JBERyyqFFxIXSXYluJA, AA, 
> AA], eligibleLeaderReplicas=null, lastKnownELR=null) for 
> topic quickstart-events3 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] partition change for 
> quickstart-events3-2 with topic ID 6ZIeidfiSTWRiOAmGEwn_g: directories: 
> [AA, AA, AA] -> 
> [AA, 7K5JBERyyqFFxIXSXYluJA, 

[jira] [Commented] (KAFKA-16162) New created topics are unavailable after upgrading to 3.7

2024-01-26 Thread Gaurav Narula (Jira)


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

Gaurav Narula commented on KAFKA-16162:
---

[~showuon] The controller's view of the logDirs is used to make leaderAndIsr 
update decisions. WIth the current fix, if a log directory fails, the 
leadership of the partitions hosted in the failed log directory will not be 
updated. I'm able to confirm this behaviour with a test.

I'll raise a PR shortly which incorporates Proven's fix and resends the 
BrokerRegistration request as well.


> New created topics are unavailable after upgrading to 3.7
> -
>
> Key: KAFKA-16162
> URL: https://issues.apache.org/jira/browse/KAFKA-16162
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 3.7.0
>Reporter: Luke Chen
>Priority: Blocker
>
> In 3.7, we introduced the KIP-858 JBOD feature, and the brokerRegistration 
> request will include the `LogDirs` fields with UUID for each log dir in each 
> broker. This info will be stored in the controller and used to identify if 
> the log dir is known and online while handling AssignReplicasToDirsRequest 
> [here|https://github.com/apache/kafka/blob/trunk/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java#L2093].
>  
> While upgrading from old version, the kafka cluster will run in 3.7 binary 
> with old metadata version, and then upgrade to newer version using 
> kafka-features.sh. That means, while brokers startup and send the 
> brokerRegistration request, it'll be using older metadata version without 
> `LogDirs` fields included. And it makes the controller has no log dir info 
> for all brokers. Later, after upgraded, if new topic is created, the flow 
> will go like this:
> 1. Controller assign replicas and adds in metadata log
> 2. brokers fetch the metadata and apply it
> 3. ReplicaManager#maybeUpdateTopicAssignment will update topic assignment
> 4. After sending ASSIGN_REPLICAS_TO_DIRS to controller with replica 
> assignment, controller will think the log dir in current replica is offline, 
> so triggering offline handler, and reassign leader to another replica, and 
> offline, until no more replicas to assign, so assigning leader to -1 (i.e. no 
> leader) 
> So, the results will be that new created topics are unavailable (with no 
> leader) because the controller thinks all log dir are offline.
> {code:java}
> lukchen@lukchen-mac kafka % bin/kafka-topics.sh --describe --topic 
> quickstart-events3 --bootstrap-server localhost:9092  
> 
> Topic: quickstart-events3 TopicId: s8s6tEQyRvmjKI6ctNTgPg PartitionCount: 
> 3   ReplicationFactor: 3Configs: segment.bytes=1073741824
>   Topic: quickstart-events3   Partition: 0Leader: none
> Replicas: 7,2,6 Isr: 6
>   Topic: quickstart-events3   Partition: 1Leader: none
> Replicas: 2,6,7 Isr: 6
>   Topic: quickstart-events3   Partition: 2Leader: none
> Replicas: 6,7,2 Isr: 6
> {code}
> The log snippet in the controller :
> {code:java}
> # handling 1st assignReplicaToDirs request
> [2024-01-18 19:34:47,370] DEBUG [QuorumController id=1] Broker 6 assigned 
> partition quickstart-events3:0 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,370] DEBUG [QuorumController id=1] Broker 6 assigned 
> partition quickstart-events3:2 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,371] DEBUG [QuorumController id=1] Broker 6 assigned 
> partition quickstart-events3:1 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] 
> offline-dir-assignment: changing partition(s): quickstart-events3-0, 
> quickstart-events3-2, quickstart-events3-1 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] partition change for 
> quickstart-events3-0 with topic ID 6ZIeidfiSTWRiOAmGEwn_g: directories: 
> [AA, AA, AA] -> 
> [7K5JBERyyqFFxIXSXYluJA, AA, AA], 
> partitionEpoch: 0 -> 1 (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] Replayed partition 
> change PartitionChangeRecord(partitionId=0, topicId=6ZIeidfiSTWRiOAmGEwn_g, 
> isr=null, leader=-2, replicas=null, removingReplicas=null, 
> addingReplicas=null, leaderRecoveryState=-1, 
> directories=[7K5JBERyyqFFxIXSXYluJA, AA, 
> AA], eligibleLeaderReplicas=null, lastKnownELR=null) for 
> topic quickstart-events3 
> 

[jira] [Commented] (KAFKA-16162) New created topics are unavailable after upgrading to 3.7

2024-01-26 Thread Luke Chen (Jira)


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

Luke Chen commented on KAFKA-16162:
---

You are right [~gnarula]. I agree the best fix should be re-sending the 
BrokerRegistration request with the same incarnation id when handling the 
metadata version update. I can see the current fix works well right now, but we 
are unsure if the logDirs in the controller view will be used for any other 
purpose in the future, which will cause another potential issue then. 
[~pprovenzano], WDYT?

> New created topics are unavailable after upgrading to 3.7
> -
>
> Key: KAFKA-16162
> URL: https://issues.apache.org/jira/browse/KAFKA-16162
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 3.7.0
>Reporter: Luke Chen
>Priority: Blocker
>
> In 3.7, we introduced the KIP-858 JBOD feature, and the brokerRegistration 
> request will include the `LogDirs` fields with UUID for each log dir in each 
> broker. This info will be stored in the controller and used to identify if 
> the log dir is known and online while handling AssignReplicasToDirsRequest 
> [here|https://github.com/apache/kafka/blob/trunk/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java#L2093].
>  
> While upgrading from old version, the kafka cluster will run in 3.7 binary 
> with old metadata version, and then upgrade to newer version using 
> kafka-features.sh. That means, while brokers startup and send the 
> brokerRegistration request, it'll be using older metadata version without 
> `LogDirs` fields included. And it makes the controller has no log dir info 
> for all brokers. Later, after upgraded, if new topic is created, the flow 
> will go like this:
> 1. Controller assign replicas and adds in metadata log
> 2. brokers fetch the metadata and apply it
> 3. ReplicaManager#maybeUpdateTopicAssignment will update topic assignment
> 4. After sending ASSIGN_REPLICAS_TO_DIRS to controller with replica 
> assignment, controller will think the log dir in current replica is offline, 
> so triggering offline handler, and reassign leader to another replica, and 
> offline, until no more replicas to assign, so assigning leader to -1 (i.e. no 
> leader) 
> So, the results will be that new created topics are unavailable (with no 
> leader) because the controller thinks all log dir are offline.
> {code:java}
> lukchen@lukchen-mac kafka % bin/kafka-topics.sh --describe --topic 
> quickstart-events3 --bootstrap-server localhost:9092  
> 
> Topic: quickstart-events3 TopicId: s8s6tEQyRvmjKI6ctNTgPg PartitionCount: 
> 3   ReplicationFactor: 3Configs: segment.bytes=1073741824
>   Topic: quickstart-events3   Partition: 0Leader: none
> Replicas: 7,2,6 Isr: 6
>   Topic: quickstart-events3   Partition: 1Leader: none
> Replicas: 2,6,7 Isr: 6
>   Topic: quickstart-events3   Partition: 2Leader: none
> Replicas: 6,7,2 Isr: 6
> {code}
> The log snippet in the controller :
> {code:java}
> # handling 1st assignReplicaToDirs request
> [2024-01-18 19:34:47,370] DEBUG [QuorumController id=1] Broker 6 assigned 
> partition quickstart-events3:0 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,370] DEBUG [QuorumController id=1] Broker 6 assigned 
> partition quickstart-events3:2 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,371] DEBUG [QuorumController id=1] Broker 6 assigned 
> partition quickstart-events3:1 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] 
> offline-dir-assignment: changing partition(s): quickstart-events3-0, 
> quickstart-events3-2, quickstart-events3-1 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] partition change for 
> quickstart-events3-0 with topic ID 6ZIeidfiSTWRiOAmGEwn_g: directories: 
> [AA, AA, AA] -> 
> [7K5JBERyyqFFxIXSXYluJA, AA, AA], 
> partitionEpoch: 0 -> 1 (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] Replayed partition 
> change PartitionChangeRecord(partitionId=0, topicId=6ZIeidfiSTWRiOAmGEwn_g, 
> isr=null, leader=-2, replicas=null, removingReplicas=null, 
> addingReplicas=null, leaderRecoveryState=-1, 
> directories=[7K5JBERyyqFFxIXSXYluJA, AA, 
> AA], eligibleLeaderReplicas=null, lastKnownELR=null) for 
> topic quickstart-events3 
> 

[jira] [Commented] (KAFKA-16162) New created topics are unavailable after upgrading to 3.7

2024-01-25 Thread Gaurav Narula (Jira)


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

Gaurav Narula commented on KAFKA-16162:
---

I think the proposed fix would never result in the controller's view of the log 
directories in the broker being updated until the broker restarts and 
re-registers.

 

This makes me wonder if a cleaner alternative would be to re-send the 
BrokerRegistration request with the same incarnation id?

 

CC: [~omnia_h_ibrahim] [~cmccabe] [~showuon] 

> New created topics are unavailable after upgrading to 3.7
> -
>
> Key: KAFKA-16162
> URL: https://issues.apache.org/jira/browse/KAFKA-16162
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 3.7.0
>Reporter: Luke Chen
>Priority: Blocker
>
> In 3.7, we introduced the KIP-858 JBOD feature, and the brokerRegistration 
> request will include the `LogDirs` fields with UUID for each log dir in each 
> broker. This info will be stored in the controller and used to identify if 
> the log dir is known and online while handling AssignReplicasToDirsRequest 
> [here|https://github.com/apache/kafka/blob/trunk/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java#L2093].
>  
> While upgrading from old version, the kafka cluster will run in 3.7 binary 
> with old metadata version, and then upgrade to newer version using 
> kafka-features.sh. That means, while brokers startup and send the 
> brokerRegistration request, it'll be using older metadata version without 
> `LogDirs` fields included. And it makes the controller has no log dir info 
> for all brokers. Later, after upgraded, if new topic is created, the flow 
> will go like this:
> 1. Controller assign replicas and adds in metadata log
> 2. brokers fetch the metadata and apply it
> 3. ReplicaManager#maybeUpdateTopicAssignment will update topic assignment
> 4. After sending ASSIGN_REPLICAS_TO_DIRS to controller with replica 
> assignment, controller will think the log dir in current replica is offline, 
> so triggering offline handler, and reassign leader to another replica, and 
> offline, until no more replicas to assign, so assigning leader to -1 (i.e. no 
> leader) 
> So, the results will be that new created topics are unavailable (with no 
> leader) because the controller thinks all log dir are offline.
> {code:java}
> lukchen@lukchen-mac kafka % bin/kafka-topics.sh --describe --topic 
> quickstart-events3 --bootstrap-server localhost:9092  
> 
> Topic: quickstart-events3 TopicId: s8s6tEQyRvmjKI6ctNTgPg PartitionCount: 
> 3   ReplicationFactor: 3Configs: segment.bytes=1073741824
>   Topic: quickstart-events3   Partition: 0Leader: none
> Replicas: 7,2,6 Isr: 6
>   Topic: quickstart-events3   Partition: 1Leader: none
> Replicas: 2,6,7 Isr: 6
>   Topic: quickstart-events3   Partition: 2Leader: none
> Replicas: 6,7,2 Isr: 6
> {code}
> The log snippet in the controller :
> {code:java}
> # handling 1st assignReplicaToDirs request
> [2024-01-18 19:34:47,370] DEBUG [QuorumController id=1] Broker 6 assigned 
> partition quickstart-events3:0 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,370] DEBUG [QuorumController id=1] Broker 6 assigned 
> partition quickstart-events3:2 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,371] DEBUG [QuorumController id=1] Broker 6 assigned 
> partition quickstart-events3:1 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] 
> offline-dir-assignment: changing partition(s): quickstart-events3-0, 
> quickstart-events3-2, quickstart-events3-1 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] partition change for 
> quickstart-events3-0 with topic ID 6ZIeidfiSTWRiOAmGEwn_g: directories: 
> [AA, AA, AA] -> 
> [7K5JBERyyqFFxIXSXYluJA, AA, AA], 
> partitionEpoch: 0 -> 1 (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] Replayed partition 
> change PartitionChangeRecord(partitionId=0, topicId=6ZIeidfiSTWRiOAmGEwn_g, 
> isr=null, leader=-2, replicas=null, removingReplicas=null, 
> addingReplicas=null, leaderRecoveryState=-1, 
> directories=[7K5JBERyyqFFxIXSXYluJA, AA, 
> AA], eligibleLeaderReplicas=null, lastKnownELR=null) for 
> topic quickstart-events3 
> 

[jira] [Commented] (KAFKA-16162) New created topics are unavailable after upgrading to 3.7

2024-01-19 Thread Proven Provenzano (Jira)


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

Proven Provenzano commented on KAFKA-16162:
---

 

Hi [~showuon] 

I think my patch will also address this other case. In step 4 the check to see 
if the log directory is offline will go through the same code path. It will see 
that there are no directories assigned to the broker and assume that it doesn't 
have any information and thus assume that the directory is online. I did run a 
manual test and everything worked for me. If you can try the code and see if 
you can produce this second case that would be really helpful. Just add it to 
this bug report. Your bug report was awesome and with the logs I was easily 
able to find the issue.

 

As for all log dirs are offline in the broker, the current behavior is that the 
broker is supposed to fail. I just want to add extra protection on the 
controller side.

 

> New created topics are unavailable after upgrading to 3.7
> -
>
> Key: KAFKA-16162
> URL: https://issues.apache.org/jira/browse/KAFKA-16162
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 3.7.0
>Reporter: Luke Chen
>Priority: Blocker
>
> In 3.7, we introduced the KIP-858 JBOD feature, and the brokerRegistration 
> request will include the `LogDirs` fields with UUID for each log dir in each 
> broker. This info will be stored in the controller and used to identify if 
> the log dir is known and online while handling AssignReplicasToDirsRequest 
> [here|https://github.com/apache/kafka/blob/trunk/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java#L2093].
>  
> While upgrading from old version, the kafka cluster will run in 3.7 binary 
> with old metadata version, and then upgrade to newer version using 
> kafka-features.sh. That means, while brokers startup and send the 
> brokerRegistration request, it'll be using older metadata version without 
> `LogDirs` fields included. And it makes the controller has no log dir info 
> for all brokers. Later, after upgraded, if new topic is created, the flow 
> will go like this:
> 1. Controller assign replicas and adds in metadata log
> 2. brokers fetch the metadata and apply it
> 3. ReplicaManager#maybeUpdateTopicAssignment will update topic assignment
> 4. After sending ASSIGN_REPLICAS_TO_DIRS to controller with replica 
> assignment, controller will think the log dir in current replica is offline, 
> so triggering offline handler, and reassign leader to another replica, and 
> offline, until no more replicas to assign, so assigning leader to -1 (i.e. no 
> leader) 
> So, the results will be that new created topics are unavailable (with no 
> leader) because the controller thinks all log dir are offline.
> {code:java}
> lukchen@lukchen-mac kafka % bin/kafka-topics.sh --describe --topic 
> quickstart-events3 --bootstrap-server localhost:9092  
> 
> Topic: quickstart-events3 TopicId: s8s6tEQyRvmjKI6ctNTgPg PartitionCount: 
> 3   ReplicationFactor: 3Configs: segment.bytes=1073741824
>   Topic: quickstart-events3   Partition: 0Leader: none
> Replicas: 7,2,6 Isr: 6
>   Topic: quickstart-events3   Partition: 1Leader: none
> Replicas: 2,6,7 Isr: 6
>   Topic: quickstart-events3   Partition: 2Leader: none
> Replicas: 6,7,2 Isr: 6
> {code}
> The log snippet in the controller :
> {code:java}
> # handling 1st assignReplicaToDirs request
> [2024-01-18 19:34:47,370] DEBUG [QuorumController id=1] Broker 6 assigned 
> partition quickstart-events3:0 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,370] DEBUG [QuorumController id=1] Broker 6 assigned 
> partition quickstart-events3:2 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,371] DEBUG [QuorumController id=1] Broker 6 assigned 
> partition quickstart-events3:1 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] 
> offline-dir-assignment: changing partition(s): quickstart-events3-0, 
> quickstart-events3-2, quickstart-events3-1 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] partition change for 
> quickstart-events3-0 with topic ID 6ZIeidfiSTWRiOAmGEwn_g: directories: 
> [AA, AA, AA] -> 
> [7K5JBERyyqFFxIXSXYluJA, AA, AA], 
> partitionEpoch: 0 -> 1 (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,372] DEBUG [QuorumController 

[jira] [Commented] (KAFKA-16162) New created topics are unavailable after upgrading to 3.7

2024-01-19 Thread Luke Chen (Jira)


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

Luke Chen commented on KAFKA-16162:
---

Thanks [~pprovenzano], I agree we also need to have a way to handle all log 
dirs are offline in a broker.
However, in this issue, I think there's also another potential bug that why 
should we update topic assignment and send ASSIGN_REPLICAS_TO_DIRS in 
`ReplicaManager#maybeUpdateTopicAssignment`?

{code:java}
1. Controller assign replicas and adds in metadata log
2. brokers fetch the metadata and apply it
3. ReplicaManager#maybeUpdateTopicAssignment will update topic assignment
4. After sending ASSIGN_REPLICAS_TO_DIRS to controller with replica assignment, 
controller will think the log dir in current replica is offline, so triggering 
offline handler, and reassign leader to another replica, and offline, until no 
more replicas to assign, so assigning leader to -1 (i.e. no leader) 
{code}

The problem still lies in the brokerRegistraion having no log dir. And when the 
controller adds the `PartitionRegistration` record, it used the info in old 
`brokerRegistraion` for the broker, so it'll get nothing for the log dir. Later 
in the broker side, when in `ReplicaManager#maybeUpdateTopicAssignment`, we'll 
find the assigned log dir UUID (which is all 0 (MIGRATING)) is different from 
the one in logManager (the real log dir UUID). I'm not sure if we can skip the 
topic assignment update if the UUID is MIGRATING and we have a real UUID? I can 
create another ticket if you think that's a separate issue. 


> New created topics are unavailable after upgrading to 3.7
> -
>
> Key: KAFKA-16162
> URL: https://issues.apache.org/jira/browse/KAFKA-16162
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 3.7.0
>Reporter: Luke Chen
>Priority: Blocker
>
> In 3.7, we introduced the KIP-858 JBOD feature, and the brokerRegistration 
> request will include the `LogDirs` fields with UUID for each log dir in each 
> broker. This info will be stored in the controller and used to identify if 
> the log dir is known and online while handling AssignReplicasToDirsRequest 
> [here|https://github.com/apache/kafka/blob/trunk/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java#L2093].
>  
> While upgrading from old version, the kafka cluster will run in 3.7 binary 
> with old metadata version, and then upgrade to newer version using 
> kafka-features.sh. That means, while brokers startup and send the 
> brokerRegistration request, it'll be using older metadata version without 
> `LogDirs` fields included. And it makes the controller has no log dir info 
> for all brokers. Later, after upgraded, if new topic is created, the flow 
> will go like this:
> 1. Controller assign replicas and adds in metadata log
> 2. brokers fetch the metadata and apply it
> 3. ReplicaManager#maybeUpdateTopicAssignment will update topic assignment
> 4. After sending ASSIGN_REPLICAS_TO_DIRS to controller with replica 
> assignment, controller will think the log dir in current replica is offline, 
> so triggering offline handler, and reassign leader to another replica, and 
> offline, until no more replicas to assign, so assigning leader to -1 (i.e. no 
> leader) 
> So, the results will be that new created topics are unavailable (with no 
> leader) because the controller thinks all log dir are offline.
> {code:java}
> lukchen@lukchen-mac kafka % bin/kafka-topics.sh --describe --topic 
> quickstart-events3 --bootstrap-server localhost:9092  
> 
> Topic: quickstart-events3 TopicId: s8s6tEQyRvmjKI6ctNTgPg PartitionCount: 
> 3   ReplicationFactor: 3Configs: segment.bytes=1073741824
>   Topic: quickstart-events3   Partition: 0Leader: none
> Replicas: 7,2,6 Isr: 6
>   Topic: quickstart-events3   Partition: 1Leader: none
> Replicas: 2,6,7 Isr: 6
>   Topic: quickstart-events3   Partition: 2Leader: none
> Replicas: 6,7,2 Isr: 6
> {code}
> The log snippet in the controller :
> {code:java}
> # handling 1st assignReplicaToDirs request
> [2024-01-18 19:34:47,370] DEBUG [QuorumController id=1] Broker 6 assigned 
> partition quickstart-events3:0 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,370] DEBUG [QuorumController id=1] Broker 6 assigned 
> partition quickstart-events3:2 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,371] DEBUG [QuorumController id=1] Broker 6 assigned 
> partition quickstart-events3:1 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,372] DEBUG 

[jira] [Commented] (KAFKA-16162) New created topics are unavailable after upgrading to 3.7

2024-01-18 Thread Proven Provenzano (Jira)


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

Proven Provenzano commented on KAFKA-16162:
---

I have a PR [GitHub Pull Request 
#15232|https://github.com/apache/kafka/pull/15232] that should fix the issue. I 
have not created tests for it yet though and it really should also add a 
condition where if a broker sends a heartbeat indicating it has no online 
directories, it should be fenced.

> New created topics are unavailable after upgrading to 3.7
> -
>
> Key: KAFKA-16162
> URL: https://issues.apache.org/jira/browse/KAFKA-16162
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 3.7.0
>Reporter: Luke Chen
>Priority: Blocker
>
> In 3.7, we introduced the KIP-858 JBOD feature, and the brokerRegistration 
> request will include the `LogDirs` fields with UUID for each log dir in each 
> broker. This info will be stored in the controller and used to identify if 
> the log dir is known and online while handling AssignReplicasToDirsRequest 
> [here|https://github.com/apache/kafka/blob/trunk/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java#L2093].
>  
> While upgrading from old version, the kafka cluster will run in 3.7 binary 
> with old metadata version, and then upgrade to newer version using 
> kafka-features.sh. That means, while brokers startup and send the 
> brokerRegistration request, it'll be using older metadata version without 
> `LogDirs` fields included. And it makes the controller has no log dir info 
> for all brokers. Later, after upgraded, if new topic is created, the flow 
> will go like this:
> 1. Controller assign replicas and adds in metadata log
> 2. brokers fetch the metadata and apply it
> 3. ReplicaManager#maybeUpdateTopicAssignment will update topic assignment
> 4. After sending ASSIGN_REPLICAS_TO_DIRS to controller with replica 
> assignment, controller will think the log dir in current replica is offline, 
> so triggering offline handler, and reassign leader to another replica, and 
> offline, until no more replicas to assign, so assigning leader to -1 (i.e. no 
> leader) 
> So, the results will be that new created topics are unavailable (with no 
> leader) because the controller thinks all log dir are offline.
> {code:java}
> lukchen@lukchen-mac kafka % bin/kafka-topics.sh --describe --topic 
> quickstart-events3 --bootstrap-server localhost:9092  
> 
> Topic: quickstart-events3 TopicId: s8s6tEQyRvmjKI6ctNTgPg PartitionCount: 
> 3   ReplicationFactor: 3Configs: segment.bytes=1073741824
>   Topic: quickstart-events3   Partition: 0Leader: none
> Replicas: 7,2,6 Isr: 6
>   Topic: quickstart-events3   Partition: 1Leader: none
> Replicas: 2,6,7 Isr: 6
>   Topic: quickstart-events3   Partition: 2Leader: none
> Replicas: 6,7,2 Isr: 6
> {code}
> The log snippet in the controller :
> {code:java}
> # handling 1st assignReplicaToDirs request
> [2024-01-18 19:34:47,370] DEBUG [QuorumController id=1] Broker 6 assigned 
> partition quickstart-events3:0 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,370] DEBUG [QuorumController id=1] Broker 6 assigned 
> partition quickstart-events3:2 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,371] DEBUG [QuorumController id=1] Broker 6 assigned 
> partition quickstart-events3:1 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] 
> offline-dir-assignment: changing partition(s): quickstart-events3-0, 
> quickstart-events3-2, quickstart-events3-1 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] partition change for 
> quickstart-events3-0 with topic ID 6ZIeidfiSTWRiOAmGEwn_g: directories: 
> [AA, AA, AA] -> 
> [7K5JBERyyqFFxIXSXYluJA, AA, AA], 
> partitionEpoch: 0 -> 1 (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] Replayed partition 
> change PartitionChangeRecord(partitionId=0, topicId=6ZIeidfiSTWRiOAmGEwn_g, 
> isr=null, leader=-2, replicas=null, removingReplicas=null, 
> addingReplicas=null, leaderRecoveryState=-1, 
> directories=[7K5JBERyyqFFxIXSXYluJA, AA, 
> AA], eligibleLeaderReplicas=null, lastKnownELR=null) for 
> topic quickstart-events3 
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-01-18 19:34:47,372] DEBUG