Browse Source
Add support for --bootstrap-controller in the following command-line tools: - kafka-cluster.sh - kafka-configs.sh - kafka-features.sh - kafka-metadata-quorum.sh To implement this, the following AdminClient APIs now support the new bootstrap.controllers configuration: - Admin.alterConfigs - Admin.describeCluster - Admin.describeConfigs - Admin.describeFeatures - Admin.describeMetadataQuorum - Admin.incrementalAlterConfigs - Admin.updateFeatures Command-line tool changes: - Add CommandLineUtils.initializeBootstrapProperties to handle parsing --bootstrap-controller in addition to --bootstrap-server. - Add --bootstrap-controller to ConfigCommand.scala, ClusterTool.java, FeatureCommand.java, and MetadataQuorumCommand.java. KafkaAdminClient changes: - Add the AdminBootstrapAddresses class to handle extracting bootstrap.servers or bootstrap.controllers from the config map for KafkaAdminClient. - In AdminMetadataManager, store the new usingBootstrapControllers boolean. Generalize authException to encompass the concept of fatal exceptions in general. (For example, the fatal exception where we talked to the wrong node type.) Treat MismatchedEndpointTypeException and UnsupportedEndpointTypeException as fatal exceptions. - Extend NodeProvider to include information about whether bootstrap.controllers is supported. - Modify the APIs described above to support bootstrap.controllers. Server-side changes: - Support DescribeConfigsRequest on kcontrollers. - Add KRaftMetadataCache to the kcontroller to simplify implemeting describeConfigs (and probably more APIs in the future). It's mainly a wrapper around MetadataImage, so there is essentially no extra resource consumption. - Split RuntimeLoggerManager out of ConfigAdminManager to handle the incrementalAlterConfigs support for BROKER_LOGGER. This is now supported on kcontrollers as well as brokers. - Fix bug in AuthHelper.computeDescribeClusterResponse that resulted in us always sending back BROKER as the endpoint type, even on the kcontroller. Miscellaneous: - Fix a few places in exceptions and log messages where we wrote "broker" instead of "node". For example, an exception in NodeApiVersions.java, and a log message in NetworkClient.java. - Fix the slf4j log prefix used by KafkaRequestHandler logging so that request handlers on a controller don't look like they're on a broker. - Make the FinalizedVersionRange constructor public for the sake of a junit test. - Add unit and integration tests for the above. Reviewers: David Arthur <mumrah@gmail.com>, Doguscan Namal <namal.doguscan@gmail.com>pull/10629/merge
Colin Patrick McCabe
1 year ago
committed by
GitHub
44 changed files with 1604 additions and 365 deletions
@ -0,0 +1,108 @@
@@ -0,0 +1,108 @@
|
||||
/* |
||||
* 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.clients.admin.internals; |
||||
|
||||
import org.apache.kafka.clients.ClientUtils; |
||||
import org.apache.kafka.clients.CommonClientConfigs; |
||||
import org.apache.kafka.clients.admin.AdminClientConfig; |
||||
import org.apache.kafka.common.config.AbstractConfig; |
||||
import org.apache.kafka.common.config.ConfigException; |
||||
|
||||
import java.net.InetSocketAddress; |
||||
import java.util.Collections; |
||||
import java.util.List; |
||||
import java.util.Objects; |
||||
|
||||
final public class AdminBootstrapAddresses { |
||||
private final boolean usingBootstrapControllers; |
||||
private final List<InetSocketAddress> addresses; |
||||
|
||||
AdminBootstrapAddresses( |
||||
boolean usingBootstrapControllers, |
||||
List<InetSocketAddress> addresses |
||||
) { |
||||
this.usingBootstrapControllers = usingBootstrapControllers; |
||||
this.addresses = addresses; |
||||
} |
||||
|
||||
public boolean usingBootstrapControllers() { |
||||
return usingBootstrapControllers; |
||||
} |
||||
|
||||
public List<InetSocketAddress> addresses() { |
||||
return addresses; |
||||
} |
||||
|
||||
public static AdminBootstrapAddresses fromConfig(AbstractConfig config) { |
||||
List<String> bootstrapServers = config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG); |
||||
if (bootstrapServers == null) { |
||||
bootstrapServers = Collections.emptyList(); |
||||
} |
||||
List<String> controllerServers = config.getList(AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG); |
||||
if (controllerServers == null) { |
||||
controllerServers = Collections.emptyList(); |
||||
} |
||||
String clientDnsLookupConfig = config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG); |
||||
if (bootstrapServers.isEmpty()) { |
||||
if (controllerServers.isEmpty()) { |
||||
throw new ConfigException("You must set either " + |
||||
CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG + " or " + |
||||
AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG); |
||||
} else { |
||||
return new AdminBootstrapAddresses(true, |
||||
ClientUtils.parseAndValidateAddresses(controllerServers, clientDnsLookupConfig)); |
||||
} |
||||
} else { |
||||
if (controllerServers.isEmpty()) { |
||||
return new AdminBootstrapAddresses(false, |
||||
ClientUtils.parseAndValidateAddresses(bootstrapServers, clientDnsLookupConfig)); |
||||
} else { |
||||
throw new ConfigException("You cannot set both " + |
||||
CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG + " and " + |
||||
AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG); |
||||
} |
||||
} |
||||
} |
||||
|
||||
@Override |
||||
public int hashCode() { |
||||
return Objects.hash(usingBootstrapControllers, addresses); |
||||
} |
||||
|
||||
@Override |
||||
public boolean equals(Object o) { |
||||
if (o == null || (!o.getClass().equals(AdminBootstrapAddresses.class))) return false; |
||||
AdminBootstrapAddresses other = (AdminBootstrapAddresses) o; |
||||
return usingBootstrapControllers == other.usingBootstrapControllers && |
||||
addresses.equals(other.addresses); |
||||
} |
||||
|
||||
@Override |
||||
public String toString() { |
||||
StringBuilder bld = new StringBuilder(); |
||||
bld.append("AdminBootstrapAddresses"); |
||||
bld.append("(usingBoostrapControllers=").append(usingBootstrapControllers); |
||||
bld.append(", addresses=["); |
||||
String prefix = ""; |
||||
for (InetSocketAddress address : addresses) { |
||||
bld.append(prefix).append(address); |
||||
prefix = ", "; |
||||
} |
||||
bld.append("])"); |
||||
return bld.toString(); |
||||
} |
||||
} |
@ -0,0 +1,81 @@
@@ -0,0 +1,81 @@
|
||||
/* |
||||
* 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.clients.admin.internals; |
||||
|
||||
import org.apache.kafka.clients.admin.AdminClientConfig; |
||||
import org.apache.kafka.common.config.ConfigException; |
||||
import org.junit.jupiter.api.Test; |
||||
import org.junit.jupiter.params.ParameterizedTest; |
||||
import org.junit.jupiter.params.provider.ValueSource; |
||||
|
||||
import java.net.InetSocketAddress; |
||||
import java.util.Arrays; |
||||
import java.util.HashMap; |
||||
import java.util.Map; |
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals; |
||||
import static org.junit.jupiter.api.Assertions.assertThrows; |
||||
|
||||
public class AdminBootstrapAddressesTest { |
||||
@ParameterizedTest |
||||
@ValueSource(booleans = {false, true}) |
||||
public void testNoBootstrapSet(boolean nullValue) { |
||||
Map<String, Object> map = new HashMap<>(); |
||||
if (nullValue) { |
||||
map.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, null); |
||||
map.put(AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG, null); |
||||
} else { |
||||
map.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, ""); |
||||
map.put(AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG, ""); |
||||
} |
||||
AdminClientConfig config = new AdminClientConfig(map); |
||||
assertEquals("You must set either bootstrap.servers or bootstrap.controllers", |
||||
assertThrows(ConfigException.class, () -> AdminBootstrapAddresses.fromConfig(config)). |
||||
getMessage()); |
||||
} |
||||
|
||||
@Test |
||||
public void testTwoBootstrapsSet() { |
||||
Map<String, Object> map = new HashMap<>(); |
||||
map.put(AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG, "localhost:9092"); |
||||
map.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); |
||||
AdminClientConfig config = new AdminClientConfig(map); |
||||
assertEquals("You cannot set both bootstrap.servers and bootstrap.controllers", |
||||
assertThrows(ConfigException.class, () -> AdminBootstrapAddresses.fromConfig(config)). |
||||
getMessage()); |
||||
} |
||||
|
||||
@ParameterizedTest |
||||
@ValueSource(booleans = {false, true}) |
||||
public void testFromConfig(boolean usingBootstrapControllers) { |
||||
Map<String, Object> map = new HashMap<>(); |
||||
String connectString = "localhost:9092,localhost:9093,localhost:9094"; |
||||
if (usingBootstrapControllers) { |
||||
map.put(AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG, connectString); |
||||
} else { |
||||
map.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, connectString); |
||||
} |
||||
AdminClientConfig config = new AdminClientConfig(map); |
||||
AdminBootstrapAddresses addresses = AdminBootstrapAddresses.fromConfig(config); |
||||
assertEquals(usingBootstrapControllers, addresses.usingBootstrapControllers()); |
||||
assertEquals(Arrays.asList( |
||||
new InetSocketAddress("localhost", 9092), |
||||
new InetSocketAddress("localhost", 9093), |
||||
new InetSocketAddress("localhost", 9094)), |
||||
addresses.addresses()); |
||||
} |
||||
} |
@ -0,0 +1,150 @@
@@ -0,0 +1,150 @@
|
||||
/* |
||||
* 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 kafka.server.logger; |
||||
|
||||
import kafka.utils.Log4jController; |
||||
import org.apache.kafka.clients.admin.AlterConfigOp.OpType; |
||||
import org.apache.kafka.common.config.LogLevelConfig; |
||||
import org.apache.kafka.common.errors.ClusterAuthorizationException; |
||||
import org.apache.kafka.common.errors.InvalidConfigurationException; |
||||
import org.apache.kafka.common.errors.InvalidRequestException; |
||||
import org.apache.kafka.common.message.IncrementalAlterConfigsRequestData.AlterConfigsResource; |
||||
import org.apache.kafka.common.message.IncrementalAlterConfigsRequestData.AlterableConfig; |
||||
import org.apache.kafka.common.protocol.Errors; |
||||
import org.slf4j.Logger; |
||||
|
||||
import java.util.ArrayList; |
||||
import java.util.Collection; |
||||
|
||||
import static org.apache.kafka.common.config.ConfigResource.Type.BROKER_LOGGER; |
||||
|
||||
/** |
||||
* Manages runtimes changes to slf4j settings. |
||||
*/ |
||||
public class RuntimeLoggerManager { |
||||
static final String VALID_LOG_LEVELS_STRING; |
||||
|
||||
static { |
||||
ArrayList<String> logLevels = new ArrayList<>(LogLevelConfig.VALID_LOG_LEVELS); |
||||
logLevels.sort(String::compareTo); |
||||
VALID_LOG_LEVELS_STRING = String.join(", ", logLevels); |
||||
} |
||||
|
||||
private final int nodeId; |
||||
private final Logger log; |
||||
|
||||
public RuntimeLoggerManager(int nodeId, Logger log) { |
||||
this.nodeId = nodeId; |
||||
this.log = log; |
||||
} |
||||
|
||||
public void applyChangesForResource( |
||||
boolean authorizedForClusterResource, |
||||
boolean validateOnly, |
||||
AlterConfigsResource resource |
||||
) { |
||||
if (!authorizedForClusterResource) { |
||||
throw new ClusterAuthorizationException(Errors.CLUSTER_AUTHORIZATION_FAILED.message()); |
||||
} |
||||
validateResourceNameIsNodeId(resource.resourceName()); |
||||
validateLogLevelConfigs(resource.configs()); |
||||
if (!validateOnly) { |
||||
alterLogLevelConfigs(resource.configs()); |
||||
} |
||||
} |
||||
|
||||
void alterLogLevelConfigs(Collection<AlterableConfig> ops) { |
||||
ops.forEach(op -> { |
||||
String loggerName = op.name(); |
||||
String logLevel = op.value(); |
||||
switch (OpType.forId(op.configOperation())) { |
||||
case SET: |
||||
if (Log4jController.logLevel(loggerName, logLevel)) { |
||||
log.warn("Updated the log level of {} to {}", loggerName, logLevel); |
||||
} else { |
||||
log.error("Failed to update the log level of {} to {}", loggerName, logLevel); |
||||
} |
||||
break; |
||||
case DELETE: |
||||
if (Log4jController.unsetLogLevel(loggerName)) { |
||||
log.warn("Unset the log level of {}", loggerName); |
||||
} else { |
||||
log.error("Failed to unset the log level of {}", loggerName); |
||||
} |
||||
break; |
||||
default: |
||||
throw new IllegalArgumentException( |
||||
"Invalid log4j configOperation: " + op.configOperation()); |
||||
} |
||||
}); |
||||
} |
||||
|
||||
void validateResourceNameIsNodeId(String resourceName) { |
||||
int requestId; |
||||
try { |
||||
requestId = Integer.parseInt(resourceName); |
||||
} catch (NumberFormatException e) { |
||||
throw new InvalidRequestException("Node id must be an integer, but it is: " + |
||||
resourceName); |
||||
} |
||||
if (requestId != nodeId) { |
||||
throw new InvalidRequestException("Unexpected node id. Expected " + nodeId + |
||||
", but received " + nodeId); |
||||
} |
||||
} |
||||
|
||||
void validateLoggerNameExists(String loggerName) { |
||||
if (!Log4jController.loggerExists(loggerName)) { |
||||
throw new InvalidConfigurationException("Logger " + loggerName + " does not exist!"); |
||||
} |
||||
} |
||||
|
||||
void validateLogLevelConfigs(Collection<AlterableConfig> ops) { |
||||
ops.forEach(op -> { |
||||
String loggerName = op.name(); |
||||
switch (OpType.forId(op.configOperation())) { |
||||
case SET: |
||||
validateLoggerNameExists(loggerName); |
||||
String logLevel = op.value(); |
||||
if (!LogLevelConfig.VALID_LOG_LEVELS.contains(logLevel)) { |
||||
throw new InvalidConfigurationException("Cannot set the log level of " + |
||||
loggerName + " to " + logLevel + " as it is not a supported log level. " + |
||||
"Valid log levels are " + VALID_LOG_LEVELS_STRING); |
||||
} |
||||
break; |
||||
case DELETE: |
||||
validateLoggerNameExists(loggerName); |
||||
if (loggerName.equals(Log4jController.ROOT_LOGGER())) { |
||||
throw new InvalidRequestException("Removing the log level of the " + |
||||
Log4jController.ROOT_LOGGER() + " logger is not allowed"); |
||||
} |
||||
break; |
||||
case APPEND: |
||||
throw new InvalidRequestException(OpType.APPEND + |
||||
" operation is not allowed for the " + BROKER_LOGGER + " resource"); |
||||
case SUBTRACT: |
||||
throw new InvalidRequestException(OpType.SUBTRACT + |
||||
" operation is not allowed for the " + BROKER_LOGGER + " resource"); |
||||
default: |
||||
throw new InvalidRequestException("Unknown operation type " + |
||||
(int) op.configOperation() + " is not alowed for the " + |
||||
BROKER_LOGGER + " resource"); |
||||
} |
||||
}); |
||||
} |
||||
} |
@ -0,0 +1,38 @@
@@ -0,0 +1,38 @@
|
||||
/* |
||||
* 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 kafka.server.metadata |
||||
|
||||
import org.apache.kafka.image.{MetadataDelta, MetadataImage} |
||||
|
||||
import org.apache.kafka.image.loader.LoaderManifest |
||||
import org.apache.kafka.image.publisher.MetadataPublisher |
||||
|
||||
class KRaftMetadataCachePublisher( |
||||
val metadataCache: KRaftMetadataCache |
||||
) extends MetadataPublisher { |
||||
override def name(): String = "KRaftMetadataCachePublisher" |
||||
|
||||
override def onMetadataUpdate( |
||||
delta: MetadataDelta, |
||||
newImage: MetadataImage, |
||||
manifest: LoaderManifest |
||||
): Unit = { |
||||
metadataCache.setImage(newImage) |
||||
} |
||||
} |
||||
|
@ -0,0 +1,98 @@
@@ -0,0 +1,98 @@
|
||||
/* |
||||
* 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 kafka.server.logger; |
||||
|
||||
import kafka.utils.Log4jController; |
||||
import org.apache.kafka.clients.admin.AlterConfigOp; |
||||
import org.apache.kafka.clients.admin.AlterConfigOp.OpType; |
||||
import org.apache.kafka.common.errors.InvalidConfigurationException; |
||||
import org.apache.kafka.common.errors.InvalidRequestException; |
||||
import org.apache.kafka.common.message.IncrementalAlterConfigsRequestData.AlterableConfig; |
||||
import org.junit.jupiter.api.Assertions; |
||||
import org.junit.jupiter.api.Test; |
||||
import org.junit.jupiter.params.ParameterizedTest; |
||||
import org.junit.jupiter.params.provider.ValueSource; |
||||
import org.slf4j.Logger; |
||||
import org.slf4j.LoggerFactory; |
||||
|
||||
import java.util.Arrays; |
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals; |
||||
|
||||
public class RuntimeLoggerManagerTest { |
||||
private final static Logger LOG = LoggerFactory.getLogger(RuntimeLoggerManagerTest.class); |
||||
|
||||
private final static RuntimeLoggerManager MANAGER = new RuntimeLoggerManager(5, LOG); |
||||
|
||||
@Test |
||||
public void testValidateSetLogLevelConfig() { |
||||
MANAGER.validateLogLevelConfigs(Arrays.asList(new AlterableConfig(). |
||||
setName(LOG.getName()). |
||||
setConfigOperation(OpType.SET.id()). |
||||
setValue("TRACE"))); |
||||
} |
||||
|
||||
@Test |
||||
public void testValidateDeleteLogLevelConfig() { |
||||
MANAGER.validateLogLevelConfigs(Arrays.asList(new AlterableConfig(). |
||||
setName(LOG.getName()). |
||||
setConfigOperation(OpType.DELETE.id()). |
||||
setValue(""))); |
||||
} |
||||
|
||||
@ParameterizedTest |
||||
@ValueSource(bytes = {(byte) 2, (byte) 3}) |
||||
public void testOperationNotAllowed(byte id) { |
||||
OpType opType = AlterConfigOp.OpType.forId(id); |
||||
assertEquals(opType + " operation is not allowed for the BROKER_LOGGER resource", |
||||
Assertions.assertThrows(InvalidRequestException.class, |
||||
() -> MANAGER.validateLogLevelConfigs(Arrays.asList(new AlterableConfig(). |
||||
setName(LOG.getName()). |
||||
setConfigOperation(id). |
||||
setValue("TRACE")))).getMessage()); |
||||
} |
||||
|
||||
@Test |
||||
public void testValidateBogusLogLevelNameNotAllowed() { |
||||
assertEquals("Cannot set the log level of " + LOG.getName() + " to BOGUS as it is not " + |
||||
"a supported log level. Valid log levels are DEBUG, ERROR, FATAL, INFO, TRACE, WARN", |
||||
Assertions.assertThrows(InvalidConfigurationException.class, |
||||
() -> MANAGER.validateLogLevelConfigs(Arrays.asList(new AlterableConfig(). |
||||
setName(LOG.getName()). |
||||
setConfigOperation(OpType.SET.id()). |
||||
setValue("BOGUS")))).getMessage()); |
||||
} |
||||
|
||||
@Test |
||||
public void testValidateSetRootLogLevelConfig() { |
||||
MANAGER.validateLogLevelConfigs(Arrays.asList(new AlterableConfig(). |
||||
setName(Log4jController.ROOT_LOGGER()). |
||||
setConfigOperation(OpType.SET.id()). |
||||
setValue("TRACE"))); |
||||
} |
||||
|
||||
@Test |
||||
public void testValidateRemoveRootLogLevelConfigNotAllowed() { |
||||
assertEquals("Removing the log level of the " + Log4jController.ROOT_LOGGER() + |
||||
" logger is not allowed", |
||||
Assertions.assertThrows(InvalidRequestException.class, |
||||
() -> MANAGER.validateLogLevelConfigs(Arrays.asList(new AlterableConfig(). |
||||
setName(Log4jController.ROOT_LOGGER()). |
||||
setConfigOperation(OpType.DELETE.id()). |
||||
setValue("")))).getMessage()); |
||||
} |
||||
} |
@ -0,0 +1,242 @@
@@ -0,0 +1,242 @@
|
||||
/* |
||||
* 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 kafka.test.server; |
||||
|
||||
import kafka.server.ControllerServer; |
||||
import kafka.testkit.KafkaClusterTestKit; |
||||
import kafka.testkit.TestKitNodes; |
||||
import org.apache.kafka.clients.admin.Admin; |
||||
import org.apache.kafka.clients.admin.AlterConfigOp; |
||||
import org.apache.kafka.clients.admin.Config; |
||||
import org.apache.kafka.clients.admin.ConfigEntry; |
||||
import org.apache.kafka.clients.admin.DescribeClusterResult; |
||||
import org.apache.kafka.clients.admin.DescribeFeaturesResult; |
||||
import org.apache.kafka.clients.admin.DescribeMetadataQuorumResult; |
||||
import org.apache.kafka.clients.admin.FeatureUpdate; |
||||
import org.apache.kafka.clients.admin.FinalizedVersionRange; |
||||
import org.apache.kafka.clients.admin.ListOffsetsResult; |
||||
import org.apache.kafka.clients.admin.OffsetSpec; |
||||
import org.apache.kafka.clients.admin.UpdateFeaturesOptions; |
||||
import org.apache.kafka.clients.admin.UpdateFeaturesResult; |
||||
import org.apache.kafka.common.TopicPartition; |
||||
import org.apache.kafka.common.config.ConfigResource; |
||||
import org.apache.kafka.common.errors.InvalidUpdateVersionException; |
||||
import org.apache.kafka.common.errors.MismatchedEndpointTypeException; |
||||
import org.apache.kafka.common.errors.UnsupportedEndpointTypeException; |
||||
import org.apache.kafka.controller.QuorumController; |
||||
import org.apache.kafka.server.common.MetadataVersion; |
||||
import org.apache.kafka.test.TestUtils; |
||||
import org.junit.jupiter.api.AfterAll; |
||||
import org.junit.jupiter.api.BeforeAll; |
||||
import org.junit.jupiter.api.Disabled; |
||||
import org.junit.jupiter.api.Test; |
||||
import org.junit.jupiter.api.TestInstance; |
||||
import org.junit.jupiter.api.Timeout; |
||||
import org.junit.jupiter.params.ParameterizedTest; |
||||
import org.junit.jupiter.params.provider.ValueSource; |
||||
|
||||
import java.util.Arrays; |
||||
import java.util.Collection; |
||||
import java.util.Collections; |
||||
import java.util.HashMap; |
||||
import java.util.Map; |
||||
import java.util.Properties; |
||||
import java.util.concurrent.ExecutionException; |
||||
import java.util.concurrent.TimeUnit; |
||||
|
||||
import static org.apache.kafka.clients.admin.AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG; |
||||
import static org.apache.kafka.clients.admin.AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG; |
||||
import static org.apache.kafka.clients.admin.ConfigEntry.ConfigSource.DYNAMIC_BROKER_CONFIG; |
||||
import static org.apache.kafka.common.config.ConfigResource.Type.BROKER; |
||||
import static org.junit.jupiter.api.Assertions.assertEquals; |
||||
import static org.junit.jupiter.api.Assertions.assertNotNull; |
||||
import static org.junit.jupiter.api.Assertions.assertThrows; |
||||
import static org.junit.jupiter.api.Assertions.assertTrue; |
||||
|
||||
@Timeout(120) |
||||
@TestInstance(TestInstance.Lifecycle.PER_CLASS) |
||||
public class BootstrapControllersIntegrationTest { |
||||
private KafkaClusterTestKit cluster; |
||||
|
||||
private String bootstrapControllerString; |
||||
|
||||
@BeforeAll |
||||
public void createCluster() throws Exception { |
||||
this.cluster = new KafkaClusterTestKit.Builder( |
||||
new TestKitNodes.Builder(). |
||||
setNumBrokerNodes(3). |
||||
setNumControllerNodes(3).build()).build(); |
||||
this.cluster.format(); |
||||
this.cluster.startup(); |
||||
this.cluster.waitForActiveController(); |
||||
this.cluster.waitForReadyBrokers(); |
||||
StringBuilder bootstrapControllerStringBuilder = new StringBuilder(); |
||||
String prefix = ""; |
||||
for (ControllerServer controller : cluster.controllers().values()) { |
||||
bootstrapControllerStringBuilder.append(prefix); |
||||
prefix = ","; |
||||
int port = controller.socketServerFirstBoundPortFuture().get(1, TimeUnit.MINUTES); |
||||
bootstrapControllerStringBuilder.append("localhost:").append(port); |
||||
} |
||||
bootstrapControllerString = bootstrapControllerStringBuilder.toString(); |
||||
} |
||||
|
||||
@AfterAll |
||||
public void destroyCluster() throws Exception { |
||||
cluster.close(); |
||||
} |
||||
|
||||
private Properties adminProperties(boolean usingBootstrapControllers) { |
||||
Properties properties = cluster.clientProperties(); |
||||
if (usingBootstrapControllers) { |
||||
properties.remove(BOOTSTRAP_SERVERS_CONFIG); |
||||
properties.setProperty(BOOTSTRAP_CONTROLLERS_CONFIG, bootstrapControllerString); |
||||
} |
||||
return properties; |
||||
} |
||||
|
||||
@Test |
||||
public void testPutBrokersInBootstrapControllersConfig() throws Exception { |
||||
Properties properties = cluster.clientProperties(); |
||||
properties.put(BOOTSTRAP_CONTROLLERS_CONFIG, properties.getProperty(BOOTSTRAP_SERVERS_CONFIG)); |
||||
properties.remove(BOOTSTRAP_SERVERS_CONFIG); |
||||
try (Admin admin = Admin.create(properties)) { |
||||
ExecutionException exception = assertThrows(ExecutionException.class, |
||||
() -> admin.describeCluster().clusterId().get(1, TimeUnit.MINUTES)); |
||||
assertNotNull(exception.getCause()); |
||||
assertEquals(MismatchedEndpointTypeException.class, exception.getCause().getClass()); |
||||
assertEquals("The request was sent to an endpoint of type BROKER, but we wanted " + |
||||
"an endpoint of type CONTROLLER", exception.getCause().getMessage()); |
||||
} |
||||
} |
||||
|
||||
@Disabled |
||||
@Test |
||||
public void testPutControllersInBootstrapBrokersConfig() throws Exception { |
||||
Properties properties = cluster.clientProperties(); |
||||
properties.put(BOOTSTRAP_SERVERS_CONFIG, bootstrapControllerString); |
||||
try (Admin admin = Admin.create(properties)) { |
||||
ExecutionException exception = assertThrows(ExecutionException.class, |
||||
() -> admin.describeCluster().clusterId().get(1, TimeUnit.MINUTES)); |
||||
assertNotNull(exception.getCause()); |
||||
assertEquals(MismatchedEndpointTypeException.class, exception.getCause().getClass()); |
||||
assertEquals("This endpoint does not appear to be a BROKER.", |
||||
exception.getCause().getMessage()); |
||||
} |
||||
} |
||||
|
||||
@ParameterizedTest |
||||
@ValueSource(booleans = {false, true}) |
||||
public void testDescribeCluster(boolean usingBootstrapControllers) throws Exception { |
||||
try (Admin admin = Admin.create(adminProperties(usingBootstrapControllers))) { |
||||
DescribeClusterResult result = admin.describeCluster(); |
||||
assertEquals(cluster.controllers().values().iterator().next().clusterId(), |
||||
result.clusterId().get(1, TimeUnit.MINUTES)); |
||||
if (usingBootstrapControllers) { |
||||
assertEquals(((QuorumController) cluster.waitForActiveController()).nodeId(), |
||||
result.controller().get().id()); |
||||
} |
||||
} |
||||
} |
||||
|
||||
@ParameterizedTest |
||||
@ValueSource(booleans = {false, true}) |
||||
public void testDescribeFeatures(boolean usingBootstrapControllers) throws Exception { |
||||
try (Admin admin = Admin.create(adminProperties(usingBootstrapControllers))) { |
||||
DescribeFeaturesResult result = admin.describeFeatures(); |
||||
short metadataVersion = cluster.controllers().values().iterator().next(). |
||||
featuresPublisher().features().metadataVersion().featureLevel(); |
||||
assertEquals(new FinalizedVersionRange(metadataVersion, metadataVersion), |
||||
result.featureMetadata().get(1, TimeUnit.MINUTES).finalizedFeatures(). |
||||
get(MetadataVersion.FEATURE_NAME)); |
||||
} |
||||
} |
||||
|
||||
@ParameterizedTest |
||||
@ValueSource(booleans = {false, true}) |
||||
public void testUpdateFeatures(boolean usingBootstrapControllers) throws Exception { |
||||
try (Admin admin = Admin.create(adminProperties(usingBootstrapControllers))) { |
||||
UpdateFeaturesResult result = admin.updateFeatures(Collections.singletonMap("foo.bar.feature", |
||||
new FeatureUpdate((short) 1, FeatureUpdate.UpgradeType.UPGRADE)), |
||||
new UpdateFeaturesOptions()); |
||||
ExecutionException exception = |
||||
assertThrows(ExecutionException.class, |
||||
() -> result.all().get(1, TimeUnit.MINUTES)); |
||||
assertNotNull(exception.getCause()); |
||||
assertEquals(InvalidUpdateVersionException.class, exception.getCause().getClass()); |
||||
assertTrue(exception.getCause().getMessage().endsWith("does not support this feature."), |
||||
"expected message to end with 'does not support this feature', but it was: " + |
||||
exception.getCause().getMessage()); |
||||
} |
||||
} |
||||
|
||||
@ParameterizedTest |
||||
@ValueSource(booleans = {false, true}) |
||||
public void testDescribeMetadataQuorum(boolean usingBootstrapControllers) throws Exception { |
||||
try (Admin admin = Admin.create(adminProperties(usingBootstrapControllers))) { |
||||
DescribeMetadataQuorumResult result = admin.describeMetadataQuorum(); |
||||
assertEquals(((QuorumController) cluster.waitForActiveController()).nodeId(), |
||||
result.quorumInfo().get(1, TimeUnit.MINUTES).leaderId()); |
||||
} |
||||
} |
||||
|
||||
@Test |
||||
public void testUsingBootstrapControllersOnUnsupportedAdminApi() throws Exception { |
||||
try (Admin admin = Admin.create(adminProperties(true))) { |
||||
ListOffsetsResult result = admin.listOffsets(Collections.singletonMap( |
||||
new TopicPartition("foo", 0), OffsetSpec.earliest())); |
||||
ExecutionException exception = |
||||
assertThrows(ExecutionException.class, |
||||
() -> result.all().get(1, TimeUnit.MINUTES)); |
||||
assertNotNull(exception.getCause()); |
||||
assertEquals(UnsupportedEndpointTypeException.class, exception.getCause().getClass()); |
||||
assertEquals("This Admin API is not yet supported when communicating directly with " + |
||||
"the controller quorum.", exception.getCause().getMessage()); |
||||
} |
||||
} |
||||
|
||||
@ParameterizedTest |
||||
@ValueSource(booleans = {false, true}) |
||||
public void testIncrementalAlterConfigs(boolean usingBootstrapControllers) throws Exception { |
||||
try (Admin admin = Admin.create(adminProperties(usingBootstrapControllers))) { |
||||
int nodeId = usingBootstrapControllers ? |
||||
cluster.controllers().values().iterator().next().config().nodeId() : |
||||
cluster.brokers().values().iterator().next().config().nodeId(); |
||||
ConfigResource nodeResource = new ConfigResource(BROKER, "" + nodeId); |
||||
ConfigResource defaultResource = new ConfigResource(BROKER, ""); |
||||
Map<ConfigResource, Collection<AlterConfigOp>> alterations = new HashMap<>(); |
||||
alterations.put(nodeResource, Arrays.asList( |
||||
new AlterConfigOp(new ConfigEntry("my.custom.config", "foo"), |
||||
AlterConfigOp.OpType.SET))); |
||||
alterations.put(defaultResource, Arrays.asList( |
||||
new AlterConfigOp(new ConfigEntry("my.custom.config", "bar"), |
||||
AlterConfigOp.OpType.SET))); |
||||
admin.incrementalAlterConfigs(alterations).all().get(1, TimeUnit.MINUTES); |
||||
TestUtils.retryOnExceptionWithTimeout(30_000, () -> { |
||||
Config config = admin.describeConfigs(Arrays.asList(nodeResource)). |
||||
all().get(1, TimeUnit.MINUTES).get(nodeResource); |
||||
ConfigEntry entry = config.entries().stream(). |
||||
filter(e -> e.name().equals("my.custom.config")). |
||||
findFirst().get(); |
||||
assertEquals(DYNAMIC_BROKER_CONFIG, entry.source(), |
||||
"Expected entry for my.custom.config to come from DYNAMIC_BROKER_CONFIG. " + |
||||
"Instead, the entry was: " + entry); |
||||
}); |
||||
} |
||||
} |
||||
} |
Loading…
Reference in new issue