jeqo commented on PR #13828: URL: https://github.com/apache/kafka/pull/13828#issuecomment-1611155345
@showuon I'm trying to test this, but TBRLMM is still complaining about missing bootstrap.servers, even when listener name is provided: ``` kafka-ts | [2023-06-28 10:19:04,131] INFO Initializing the resources. (org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManager) kafka-ts | [2023-06-28 10:19:04,141] ERROR Uncaught exception in thread 'RLMMInitializationThread': (org.apache.kafka.common.utils.KafkaThread) kafka-ts | org.apache.kafka.common.config.ConfigException: Missing required configuration "bootstrap.servers" which has no default value. kafka-ts | at org.apache.kafka.common.config.ConfigDef.parseValue(ConfigDef.java:496) kafka-ts | at org.apache.kafka.common.config.ConfigDef.parse(ConfigDef.java:486) kafka-ts | at org.apache.kafka.common.config.AbstractConfig.<init>(AbstractConfig.java:112) kafka-ts | at org.apache.kafka.common.config.AbstractConfig.<init>(AbstractConfig.java:145) kafka-ts | at org.apache.kafka.clients.admin.AdminClientConfig.<init>(AdminClientConfig.java:244) kafka-ts | at org.apache.kafka.clients.admin.Admin.create(Admin.java:144) kafka-ts | at org.apache.kafka.clients.admin.AdminClient.create(AdminClient.java:49) kafka-ts | at org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManager.initializeResources(TopicBasedRemoteLogMetadataManager.java:366) kafka-ts | at org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManager.lambda$configure$1(TopicBasedRemoteLogMetadataManager.java:352) kafka-ts | at java.base/java.lang.Thread.run(Thread.java:829) ``` Looking at the code, I can see listener name being passed, ``` kafka-ts | remote.log.metadata.manager.class.name = org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManager kafka-ts | remote.log.metadata.manager.class.path = null kafka-ts | remote.log.metadata.manager.impl.prefix = rlmm.config. kafka-ts | remote.log.metadata.manager.listener.name = BROKER ``` but when initializing the resources, properties without the right prefix are ignored: https://github.com/apache/kafka/blob/f32ebeab17ce574660669873402a7f40927d0492/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerConfig.java#L136-L159 Let me know if I'm reading this properly to create an issue, otherwise I may be missing something. Many thanks! -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org