New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
KAFKA-15066: add "remote.log.metadata.manager.listener.name" config to rlmm #13828
Conversation
private void configureRLMM() { | ||
final Map<String, Object> rlmmProps = new HashMap<>(rlmConfig.remoteLogMetadataManagerProps()); | ||
|
||
rlmmProps.put(KafkaConfig.BrokerIdProp(), brokerId); | ||
rlmmProps.put(KafkaConfig.LogDirProp(), logDir); | ||
endpoint.ifPresent(e -> { | ||
rlmmProps.put("bootstrap.servers", e.host() + ":" + e.port()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
please use the constant CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, but that would add dependency with client
module with core
. That's why I use plain string instead
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ok
private void configureRLMM() { | ||
final Map<String, Object> rlmmProps = new HashMap<>(rlmConfig.remoteLogMetadataManagerProps()); | ||
|
||
rlmmProps.put(KafkaConfig.BrokerIdProp(), brokerId); | ||
rlmmProps.put(KafkaConfig.LogDirProp(), logDir); | ||
endpoint.ifPresent(e -> { | ||
rlmmProps.put("bootstrap.servers", e.host() + ":" + e.port()); | ||
rlmmProps.put("security.protocol", e.securityProtocol().name); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
please use the constant CommonClientConfigs.SECURITY_PROTOCOL_CONFIG
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
One last question Luke, should a unit test have failed when cluster.id property was missing in first revision? It probably hints towards a gap in testing. Can we add that? (or it's ok to say it will be added when Satish adds rest of the tests)
clients/src/main/java/org/apache/kafka/common/network/ListenerName.java
Outdated
Show resolved
Hide resolved
The test is added in the last commit: 39de62c Also, I reverted the Lines 45 to 48 in 513e1c6
It's saying this is not a required config. Only used when |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
-
We probably want to update the KIP-405 here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-405%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Configs and specify that this config is optional
-
From the javadoc of RLMM
When this is configured all other
required properties can be passed as properties with prefix of 'remote.log.metadata.manager.listener
Can we please add a test to verify this? (asking because while constructing the rlmmProps, we don't pass any other configs with the listener prefix)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks @showuon for adding the missing integration of starting RLMM with the local endpoints of the broker. Had an initial pass of the PR, left a minor comment.
@divijvaidya @satishd , PR updated. Thanks.
Updated.
I think we don't have this implemented. We should pass Thanks. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for patiently working through all the comments Luke! Looks good to me.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks @showuon for addressing the review comments. LGTM.
@showuon This is no more valid, KIP needs to be updated with the prefix based configs for RSM and RLMM. Will update the KIP with those details. |
Good to know, thanks Satish. I've assigned KAFKA-15083 to you. You can close it once you've updated the KIP. |
Still cannot get a healthy CI build results. Try to rebase to the latest trunk. |
Failed tests are unrelated and the failed
|
@showuon I'm trying to test this, but TBRLMM is still complaining about missing bootstrap.servers, even when listener name is provided:
Looking at the code, I can see listener name being passed,
but when initializing the resources, properties without the right prefix are ignored: Lines 136 to 159 in f32ebea
Let me know if I'm reading this properly to create an issue, otherwise I may be missing something (is this supposed to be handled by https://issues.apache.org/jira/browse/KAFKA-15083). Many thanks! |
@jeqo , thanks for raising it. Yes, I forgot to pass the |
Sure!, created this: #13938 |
add "remote.log.metadata.manager.listener.name" config to rlmm to allow producer/consumer to connect to the server. Also add tests.
Committer Checklist (excluded from commit message)