Skip to content

Commit a94f7ca

Browse files
MINOR: add lower case lister name integration test (#19932)
In [KIP-1143](https://cwiki.apache.org/confluence/x/LwqWF), it deprecated Endpoint#listenerName and removed org.apache.kafka.network.EndPoint. Certain parts of the code depend on listener name normalization. We should add a test to make sure there is no regression. Followup: https: //github.com//pull/19191#issuecomment-2939855317 Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
1 parent 6f946d5 commit a94f7ca

File tree

3 files changed

+42
-2
lines changed

3 files changed

+42
-2
lines changed

core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala

Lines changed: 27 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -43,7 +43,7 @@ import org.apache.kafka.metadata.bootstrap.BootstrapMetadata
4343
import org.apache.kafka.network.SocketServerConfigs
4444
import org.apache.kafka.server.authorizer._
4545
import org.apache.kafka.server.common.{ApiMessageAndVersion, KRaftVersion, MetadataVersion}
46-
import org.apache.kafka.server.config.{KRaftConfigs, ServerConfigs}
46+
import org.apache.kafka.server.config.{KRaftConfigs, ReplicationConfigs, ServerConfigs}
4747
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
4848
import org.apache.kafka.server.quota
4949
import org.apache.kafka.server.quota.{ClientQuotaCallback, ClientQuotaType}
@@ -130,6 +130,32 @@ class KRaftClusterTest {
130130
}
131131
}
132132

133+
@Test
134+
def testClusterWithLowerCaseListeners(): Unit = {
135+
Using.resource(new KafkaClusterTestKit.Builder(
136+
new TestKitNodes.Builder().
137+
setNumBrokerNodes(1).
138+
setBrokerListenerName(new ListenerName("external")).
139+
setNumControllerNodes(3).
140+
build()).build()
141+
) { cluster =>
142+
cluster.format()
143+
cluster.startup()
144+
cluster.brokers().forEach((_, broker) => {
145+
assertEquals("external://localhost:0", broker.config.get(SocketServerConfigs.LISTENERS_CONFIG))
146+
assertEquals("external", broker.config.get(ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG))
147+
assertEquals("external:PLAINTEXT,CONTROLLER:PLAINTEXT", broker.config.get(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG))
148+
})
149+
TestUtils.waitUntilTrue(() => cluster.brokers().get(0).brokerState == BrokerState.RUNNING,
150+
"Broker never made it to RUNNING state.")
151+
TestUtils.waitUntilTrue(() => cluster.raftManagers().get(0).client.leaderAndEpoch().leaderId.isPresent,
152+
"RaftManager was not initialized.")
153+
Using.resource(Admin.create(cluster.clientProperties())) { admin =>
154+
assertEquals(cluster.nodes().clusterId(), admin.describeCluster().clusterId().get())
155+
}
156+
}
157+
}
158+
133159
@Test
134160
def testCreateClusterAndWaitForBrokerInRunningState(): Unit = {
135161
val cluster = new KafkaClusterTestKit.Builder(

core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1869,4 +1869,12 @@ class KafkaConfigTest {
18691869
props.put(ShareGroupConfig.SHARE_GROUP_RECORD_LOCK_DURATION_MS_CONFIG, "30000")
18701870
assertDoesNotThrow(() => KafkaConfig.fromProps(props))
18711871
}
1872+
1873+
@Test
1874+
def testLowercaseControllerListenerNames(): Unit = {
1875+
val props = createDefaultConfig()
1876+
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "controller")
1877+
val message = assertThrows(classOf[IllegalArgumentException], () => KafkaConfig.fromProps(props)).getMessage
1878+
assertEquals("requirement failed: controller.listener.names must contain at least one value appearing in the 'listeners' configuration when running the KRaft controller role", message)
1879+
}
18721880
}

test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/KafkaClusterTestKit.java

Lines changed: 7 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -556,7 +556,10 @@ public String bootstrapServers() {
556556
int brokerId = entry.getKey();
557557
BrokerServer broker = entry.getValue();
558558
ListenerName listenerName = nodes.brokerListenerName();
559-
int port = broker.boundPort(listenerName);
559+
// The KafkaConfig#listeners method normalizes the listener name.
560+
// The result from TestKitNodes#brokerListenerName method should be normalized as well,
561+
// so that it matches the listener name in the KafkaConfig.
562+
int port = broker.boundPort(ListenerName.normalised(listenerName.value()));
560563
if (port <= 0) {
561564
throw new RuntimeException("Broker " + brokerId + " does not yet " +
562565
"have a bound port for " + listenerName + ". Did you start " +
@@ -575,6 +578,9 @@ public String bootstrapControllers() {
575578
int id = entry.getKey();
576579
ControllerServer controller = entry.getValue();
577580
ListenerName listenerName = nodes.controllerListenerName();
581+
// Although the KafkaConfig#listeners method normalizes the listener name,
582+
// the controller.listener.names configuration does not allow lowercase input,
583+
// so there is no lowercase controller listener name, and we don't need to normalize it.
578584
int port = controller.socketServer().boundPort(listenerName);
579585
if (port <= 0) {
580586
throw new RuntimeException("Controller " + id + " does not yet " +

0 commit comments

Comments
 (0)