[
https://issues.apache.org/jira/browse/KAFKA-18856?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17929864#comment-17929864
]
黃竣陽 commented on KAFKA-18856:
-----------------------------
I conducted the following test on a 3.9 broker joining a 4.0 controller. If we
format Kafka 4.0 with MetadataVersion 3.9-IV0, the 3.9 controller can
successfully join the 4.0 controller. The steps are
Kafka 4.0 controller
{code:java}
bin/kafka-storage.sh format --release-version 3.9-IV0 --standalone -t
$KAFKA_CLUSTER_ID -c config/server.properties
bin/kafka-server-start.sh config/server.properties{code}
Kafka 3.9 broker
{code:java}
bin/kafka-storage.sh format --no-initial-controllers -t 9uyUnSE9T46OAm4rhlmpRw
-c config/kraft/broker.properties
bin/kafka-server-start.sh config/kraft/broker.properties {code}
The result is
{code:java}
./bin/kafka-metadata-quorum.sh --bootstrap-controller localhost:9093 describe
--status
ClusterId: 9uyUnSE9T46OAm4rhlmpRw
LeaderId: 1
LeaderEpoch: 1
HighWatermark: 103
MaxFollowerLag: 0
MaxFollowerLagTimeMs: 0
CurrentVoters: [{"id": 1, "directoryId": "zax3CYNFS9mGt8b_2DpoOw",
"endpoints": ["CONTROLLER://localhost:9093"]}]
CurrentObservers: [{"id": 3, "directoryId":
"bKQBgNvIHRzLQ7a8d68WfA"}]{code}
but I use MetadataVersion 4.0, I also have the same error message, The steps are
Kafka 4.0 controller
{code:java}
bin/kafka-storage.sh format --standalone -t $KAFKA_CLUSTER_ID -c
config/server.properties
bin/kafka-server-start.sh config/server.properties{code}
Kafka 3.9 broker
{code:java}
bin/kafka-storage.sh format --no-initial-controllers -t 9uyUnSE9T46OAm4rhlmpRw
-c config/kraft/broker.properties
bin/kafka-server-start.sh config/kraft/broker.properties {code}
Error message is
{code:java}
[2025-02-25 01:05:06,530] ERROR Encountered metadata loading fault: Unhandled
error initializing new publishers
(org.apache.kafka.server.fault.LoggingFaultHandler)
org.apache.kafka.image.writer.UnwritableMetadataException: Metadata has been
lost because the following could not be represented in metadata.version
3.0-IV1: feature flag(s): group.version, transaction.version {code}
> 3.9 broker unable to join dynamic quorum with 4.0 controller
> ------------------------------------------------------------
>
> Key: KAFKA-18856
> URL: https://issues.apache.org/jira/browse/KAFKA-18856
> Project: Kafka
> Issue Type: Bug
> Reporter: TengYao Chi
> Assignee: TengYao Chi
> Priority: Major
>
> Test step:
> * format 4.0 controller /w controller.properties & standalone
> ** bin/kafka-storage.sh format --standalone -t "dAtBluZORiuJEKV2uSWB2Q" -c
> config/controller.properties
> * start 4.0 controller
> * init 3.9 broker /w kraft/broker.properties & no-initial-controllers
> ** bin/kafka-storage.sh format -t "dAtBluZORiuJEKV2uSWB2Q" --config
> config/kraft/broker.properties -N
> * start 3.9 broker
> error message:
> {code:java}
> [2025-02-24 23:13:21,271] ERROR Encountered metadata loading fault: Unhandled
> error initializing new publishers
> (org.apache.kafka.server.fault.LoggingFaultHandler)
> org.apache.kafka.image.writer.UnwritableMetadataException: Metadata has been
> lost because the following could not be represented in metadata.version
> 3.0-IV1: feature flag(s): group.version, transaction.version
> at
> org.apache.kafka.image.writer.ImageWriterOptions.handleLoss(ImageWriterOptions.java:94)
> at
> org.apache.kafka.image.FeaturesImage.handleFeatureLevelNotSupported(FeaturesImage.java:108)
> at org.apache.kafka.image.FeaturesImage.write(FeaturesImage.java:88)
> at org.apache.kafka.image.MetadataImage.write(MetadataImage.java:153)
> at
> org.apache.kafka.image.loader.MetadataLoader.initializeNewPublishers(MetadataLoader.java:296)
> at
> org.apache.kafka.image.loader.MetadataLoader.lambda$scheduleInitializeNewPublishers$0(MetadataLoader.java:267)
> at
> org.apache.kafka.queue.KafkaEventQueue$EventContext.run(KafkaEventQueue.java:132)
> at
> org.apache.kafka.queue.KafkaEventQueue$EventHandler.handleEvents(KafkaEventQueue.java:215)
> at
> org.apache.kafka.queue.KafkaEventQueue$EventHandler.run(KafkaEventQueue.java:186)
> at java.base/java.lang.Thread.run(Thread.java:1583)
> ^C[2025-02-24 23:13:21,658] INFO Terminating process due to signal SIGINT
> (org.apache.kafka.common.utils.LoggingSignalHandler)
> [2025-02-24 23:13:21,660] INFO App info kafka.server for 2 unregistered
> (org.apache.kafka.common.utils.AppInfoParser) {code}
--
This message was sent by Atlassian Jira
(v8.20.10#820010)