jsancio commented on code in PR #16774:
URL: https://github.com/apache/kafka/pull/16774#discussion_r1707498582
##########
clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java:
##########
@@ -4667,6 +4667,9 @@ AddRaftVoterRequest.Builder createRequest(int timeoutMs) {
setPort(endpoint.port())));
return new AddRaftVoterRequest.Builder(
new AddRaftVoterRequestData().
+ setClusterId(options.clusterId().isPresent() ?
+ options.clusterId().get() : null).
Review Comment:
This is the same as
```java
setClusterId(options.clusterId().orElse(null)).
```
##########
tools/src/main/java/org/apache/kafka/tools/MetadataQuorumCommand.java:
##########
@@ -291,4 +320,179 @@ public String toString() {
return sb.toString();
}
}
+
+ private static void addAddControllerSubParser(Subparsers subparsers) {
+ Subparser addControllerParser = subparsers
+ .addParser("add-controller")
+ .help("Add a controller to the KRaft controller cluster");
+
+ addControllerParser
+ .addArgument("--dry-run")
+ .help("True if we should print what would be done, but not do it.")
+ .action(Arguments.storeTrue());
+ }
+
+ static int getControllerId(Properties props) throws TerseException {
+ if (!props.containsKey(KRaftConfigs.NODE_ID_CONFIG)) {
+ throw new TerseException(KRaftConfigs.NODE_ID_CONFIG + " not found
in configuration " +
+ "file. Is this a valid controller configuration file?");
+ }
+ int nodeId =
Integer.parseInt(props.getProperty(KRaftConfigs.NODE_ID_CONFIG));
+ if (nodeId < 0) {
+ throw new TerseException(KRaftConfigs.NODE_ID_CONFIG + " was
negative in configuration " +
+ "file. Is this a valid controller configuration file?");
+ }
+ if (!props.getOrDefault(KRaftConfigs.PROCESS_ROLES_CONFIG,
"").toString().contains("controller")) {
+ throw new TerseException(KRaftConfigs.PROCESS_ROLES_CONFIG + " did
not contain 'controller' in " +
+ "configuration file. Is this a valid controller configuration
file?");
+ }
+ return nodeId;
+ }
+
+ static String getMetadataDirectory(Properties props) throws TerseException
{
+ if (props.containsKey(KRaftConfigs.METADATA_LOG_DIR_CONFIG)) {
+ return props.getProperty(KRaftConfigs.METADATA_LOG_DIR_CONFIG);
+ }
+ if (props.containsKey(ServerLogConfigs.LOG_DIRS_CONFIG)) {
+ String[] logDirs =
props.getProperty(ServerLogConfigs.LOG_DIRS_CONFIG).trim().split(",");
+ if (logDirs.length > 0) {
+ return logDirs[0];
+ }
+ }
+ throw new TerseException("Neither " +
KRaftConfigs.METADATA_LOG_DIR_CONFIG + " nor " +
+ ServerLogConfigs.LOG_DIRS_CONFIG + " were found. Is this a valid
controller " +
+ "configuration file?");
+ }
+
+ static Uuid getMetadataDirectoryId(String metadataDirectory) throws
Exception {
+ MetaPropertiesEnsemble ensemble = new MetaPropertiesEnsemble.Loader().
+ addLogDirs(Collections.singletonList(metadataDirectory)).
+ addMetadataLogDir(metadataDirectory).
+ load();
+ MetaProperties metaProperties =
ensemble.logDirProps().get(metadataDirectory);
+ if (metaProperties == null) {
+ throw new TerseException("Unable to read meta.properties from " +
metadataDirectory);
+ }
+ if (!metaProperties.directoryId().isPresent()) {
+ throw new TerseException("No directory id found in " +
metadataDirectory);
+ }
+ return metaProperties.directoryId().get();
+ }
+
+ static Set<RaftVoterEndpoint> getControllerAdvertisedListeners(
+ Properties props
+ ) throws Exception {
+ Map<String, Endpoint> listeners = new HashMap<>();
+ SocketServerConfigs.listenerListToEndPoints(
+ props.getOrDefault(SocketServerConfigs.LISTENERS_CONFIG,
"").toString(),
+ __ -> SecurityProtocol.PLAINTEXT).forEach(e ->
listeners.put(e.listenerName().get(), e));
+ SocketServerConfigs.listenerListToEndPoints(
+
props.getOrDefault(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG,
"").toString(),
+ __ -> SecurityProtocol.PLAINTEXT).forEach(e ->
listeners.put(e.listenerName().get(), e));
+ if (!props.containsKey(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG))
{
+ throw new
TerseException(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG +
+ " was not found. Is this a valid controller configuration
file?");
+ }
+ LinkedHashSet<RaftVoterEndpoint> results = new LinkedHashSet<>();
+ for (String listenerName : props.getProperty(
+ KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG).split(",")) {
+ listenerName = ListenerName.normalised(listenerName).value();
+ Endpoint endpoint = listeners.get(listenerName);
+ if (endpoint == null) {
+ throw new TerseException("Cannot find information about
controller listener name: " +
+ listenerName);
+ }
+ results.add(new RaftVoterEndpoint(endpoint.listenerName().get(),
+ endpoint.host() == null ? "localhost" : endpoint.host(),
Review Comment:
Should it be an error if the host is `null`? Do we assume localhost for
brokers if they don't specify a host in the listeners or advertise listeners
configuration?
##########
clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java:
##########
@@ -4711,6 +4714,8 @@ public RemoveRaftVoterResult removeRaftVoter(
RemoveRaftVoterRequest.Builder createRequest(int timeoutMs) {
return new RemoveRaftVoterRequest.Builder(
new RemoveRaftVoterRequestData().
+ setClusterId(options.clusterId().isPresent() ?
+ options.clusterId().get() : null).
Review Comment:
This is the same as
```java
setClusterId(options.clusterId().orElse(null)).
```
##########
server/src/test/java/org/apache/kafka/network/SocketServerConfigsTest.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.network;
+
+import org.apache.kafka.common.Endpoint;
+import org.apache.kafka.common.network.ListenerName;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class SocketServerConfigsTest {
+ @Test
+ public void testDefaultNameToSecurityProto() {
+ Map<ListenerName, SecurityProtocol> expected = new HashMap<>();
+ expected.put(new ListenerName("PLAINTEXT"),
SecurityProtocol.PLAINTEXT);
+ expected.put(new ListenerName("SSL"), SecurityProtocol.SSL);
+ expected.put(new ListenerName("SASL_PLAINTEXT"),
SecurityProtocol.SASL_PLAINTEXT);
+ expected.put(new ListenerName("SASL_SSL"), SecurityProtocol.SASL_SSL);
+ assertEquals(expected,
SocketServerConfigs.DEFAULT_NAME_TO_SECURITY_PROTO);
+ }
+
+ @Test
+ public void testListenerListToEndPointsWithEmptyString() {
+ assertEquals(Arrays.asList(),
+ SocketServerConfigs.listenerListToEndPoints("",
+ SocketServerConfigs.DEFAULT_NAME_TO_SECURITY_PROTO));
+ }
+
+ @Test
+ public void testListenerListToEndPointsWithBlankString() {
+ assertEquals(Arrays.asList(),
+ SocketServerConfigs.listenerListToEndPoints(" ",
+ SocketServerConfigs.DEFAULT_NAME_TO_SECURITY_PROTO));
+ }
+
+ @Test
+ public void testListenerListToEndPointsWithOneEndpoint() {
+ assertEquals(Arrays.asList(
+ new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT,
"example.com", 8080)),
+
SocketServerConfigs.listenerListToEndPoints("PLAINTEXT://example.com:8080",
+ SocketServerConfigs.DEFAULT_NAME_TO_SECURITY_PROTO));
+ }
+
+ // Regression test for KAFKA-3719
+ @Test
+ public void testListenerListToEndPointsWithUnderscores() {
+ assertEquals(Arrays.asList(
+ new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT,
"example.com", 8080),
+ new Endpoint("SSL", SecurityProtocol.SSL, "local_host", 8081)),
+
SocketServerConfigs.listenerListToEndPoints("PLAINTEXT://example.com:8080,SSL://local_host:8081",
+ SocketServerConfigs.DEFAULT_NAME_TO_SECURITY_PROTO));
+ }
+
+ @Test
+ public void testListenerListToEndPointsWithWildcard() {
+ assertEquals(Arrays.asList(
+ new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, null, 8080)),
+
SocketServerConfigs.listenerListToEndPoints("PLAINTEXT://:8080",
+ SocketServerConfigs.DEFAULT_NAME_TO_SECURITY_PROTO));
+ }
+
+ @Test
+ public void testListenerListToEndPointsWithIpV6() {
+ assertEquals(Arrays.asList(
+ new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "::1",
9092)),
+
SocketServerConfigs.listenerListToEndPoints("PLAINTEXT://[::1]:9092",
+ SocketServerConfigs.DEFAULT_NAME_TO_SECURITY_PROTO));
+ }
+
+ @Test
+ public void testAnotherListenerListToEndPointsWithIpV6() {
+ assertEquals(Arrays.asList(
+ new Endpoint("SASL_SSL", SecurityProtocol.SASL_SSL,
"fe80::b1da:69ca:57f7:63d8%3", 9092)),
+
SocketServerConfigs.listenerListToEndPoints("SASL_SSL://[fe80::b1da:69ca:57f7:63d8%3]:9092",
+ SocketServerConfigs.DEFAULT_NAME_TO_SECURITY_PROTO));
Review Comment:
Interesting that the character `%` is allowed in IPv6:
https://superuser.com/a/99753
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]