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

Jianbin Chen commented on KAFKA-16662:
--------------------------------------

I have encountered the same issue. Can anyone help me with this? I upgraded 
from 3.5.1 to 3.7.0, and I have already changed inter.broker.protocol.version 
to 3.7 and ran it for some time.

But I have never executed

`./bin/kafka-features.sh upgrade --metadata 3.7`

The last time I restarted the cluster, I found that it could not be started 
anymore. The last line of the log is as follows:

```

[2024-05-22 11:01:41,087] INFO [MetadataLoader id=3] 
maybePublishMetadata(LOG_DELTA): The loader is still catching up because we 
have loaded up to offset 41872530, but the high water mark is 41872532 
(org.apache.kafka.image.loader.MetadataLoader)
[2024-05-22 11:01:41,088] INFO [MetadataLoader id=3] 
maybePublishMetadata(LOG_DELTA): The loader finished catching up to the current 
high water mark of 41872532 (org.apache.kafka.image.loader.MetadataLoader)
[2024-05-22 11:01:41,092] INFO [BrokerLifecycleManager id=3] The broker has 
caught up. Transitioning from STARTING to RECOVERY. 
(kafka.server.BrokerLifecycleManager)
[2024-05-22 11:01:41,092] INFO [BrokerServer id=3] Finished waiting for the 
controller to acknowledge that we are caught up (kafka.server.BrokerServer)
[2024-05-22 11:01:41,092] INFO [BrokerServer id=3] Waiting for the initial 
broker metadata update to be published (kafka.server.BrokerServer)
[2024-05-22 11:01:41,095] ERROR Encountered fatal fault: Unhandled error 
initializing new publishers 
(org.apache.kafka.server.fault.ProcessTerminatingFaultHandler)
org.apache.kafka.image.writer.UnwritableMetadataException: Metadata has been 
lost because the following could not be represented in metadata version 
3.5-IV2: the directory assignment state of one or more replicas
    at 
org.apache.kafka.image.writer.ImageWriterOptions.handleLoss(ImageWriterOptions.java:94)
    at 
org.apache.kafka.metadata.PartitionRegistration.toRecord(PartitionRegistration.java:391)
    at org.apache.kafka.image.TopicImage.write(TopicImage.java:71)
    at org.apache.kafka.image.TopicsImage.write(TopicsImage.java:84)
    at org.apache.kafka.image.MetadataImage.write(MetadataImage.java:155)
    at 
org.apache.kafka.image.loader.MetadataLoader.initializeNewPublishers(MetadataLoader.java:295)
    at 
org.apache.kafka.image.loader.MetadataLoader.lambda$scheduleInitializeNewPublishers$0(MetadataLoader.java:266)
    at 
org.apache.kafka.queue.KafkaEventQueue$EventContext.run(KafkaEventQueue.java:127)
    at 
org.apache.kafka.queue.KafkaEventQueue$EventHandler.handleEvents(KafkaEventQueue.java:210)
    at 
org.apache.kafka.queue.KafkaEventQueue$EventHandler.run(KafkaEventQueue.java:181)
    at java.base/java.lang.Thread.run(Thread.java:1583)

```

> UnwritableMetadataException: Metadata has been lost
> ---------------------------------------------------
>
>                 Key: KAFKA-16662
>                 URL: https://issues.apache.org/jira/browse/KAFKA-16662
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 3.7.0
>         Environment: Docker Image (bitnami/kafka:3.7.0)
> via Docker Compose
>            Reporter: Tobias Bohn
>            Priority: Major
>         Attachments: log.txt
>
>
> Hello,
> First of all: I am new to this Jira and apologize if anything is set or 
> specified incorrectly. Feel free to advise me.
> We currently have an error in our test system, which unfortunately I can't 
> solve, because I couldn't find anything related to it. No solution could be 
> found via the mailing list either.
> The error occurs when we want to start up a node. The node runs using Kraft 
> and is both a controller and a broker. The following error message appears at 
> startup:
> {code:java}
> kafka  | [2024-04-16 06:18:13,707] ERROR Encountered fatal fault: Unhandled 
> error initializing new publishers 
> (org.apache.kafka.server.fault.ProcessTerminatingFaultHandler)
> kafka  | org.apache.kafka.image.writer.UnwritableMetadataException: Metadata 
> has been lost because the following could not be represented in metadata 
> version 3.5-IV2: the directory assignment state of one or more replicas
> kafka  |        at 
> org.apache.kafka.image.writer.ImageWriterOptions.handleLoss(ImageWriterOptions.java:94)
> kafka  |        at 
> org.apache.kafka.metadata.PartitionRegistration.toRecord(PartitionRegistration.java:391)
> kafka  |        at org.apache.kafka.image.TopicImage.write(TopicImage.java:71)
> kafka  |        at 
> org.apache.kafka.image.TopicsImage.write(TopicsImage.java:84)
> kafka  |        at 
> org.apache.kafka.image.MetadataImage.write(MetadataImage.java:155)
> kafka  |        at 
> org.apache.kafka.image.loader.MetadataLoader.initializeNewPublishers(MetadataLoader.java:295)
> kafka  |        at 
> org.apache.kafka.image.loader.MetadataLoader.lambda$scheduleInitializeNewPublishers$0(MetadataLoader.java:266)
> kafka  |        at 
> org.apache.kafka.queue.KafkaEventQueue$EventContext.run(KafkaEventQueue.java:127)
> kafka  |        at 
> org.apache.kafka.queue.KafkaEventQueue$EventHandler.handleEvents(KafkaEventQueue.java:210)
> kafka  |        at 
> org.apache.kafka.queue.KafkaEventQueue$EventHandler.run(KafkaEventQueue.java:181)
> kafka  |        at java.base/java.lang.Thread.run(Thread.java:840)
> kafka exited with code 0 {code}
> We use Docker to operate the cluster. The error occurred while we were trying 
> to restart a node. All other nodes in the cluster are still running correctly.
> If you need further information, please let us know. The complete log is 
> attached to this issue.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to