clarkwtc commented on code in PR #19187: URL: https://github.com/apache/kafka/pull/19187#discussion_r1991935626
########## core/src/test/java/kafka/test/api/AdminClientRebootstrapTest.java: ########## @@ -20,91 +20,85 @@ import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.test.ClusterInstance; -import org.apache.kafka.common.test.api.ClusterConfig; -import org.apache.kafka.common.test.api.ClusterTemplate; +import org.apache.kafka.common.test.api.ClusterConfigProperty; +import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.Type; import org.apache.kafka.coordinator.group.GroupCoordinatorConfig; import org.apache.kafka.test.TestUtils; -import java.util.HashMap; +import java.time.Duration; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.TimeUnit; -import java.util.stream.Stream; +import java.util.concurrent.TimeoutException; + +import static org.junit.jupiter.api.Assertions.assertThrows; public class AdminClientRebootstrapTest { - private static final int BROKER_COUNT = 2; - - private static List<ClusterConfig> generator() { - // Enable unclean leader election for the test topic - Map<String, String> serverProperties = Map.of( - TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, "true", - GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, String.valueOf(BROKER_COUNT) - ); - - return Stream.of(false, true) - .map(AdminClientRebootstrapTest::getRebootstrapConfig) - .map(rebootstrapProperties -> AdminClientRebootstrapTest.buildConfig(serverProperties, rebootstrapProperties)) - .toList(); - } + private static final String TOPIC = "topic"; + private static final int PARTITIONS = 2; - private static Map<String, String> getRebootstrapConfig(boolean useRebootstrapTriggerMs) { - Map<String, String> properties = new HashMap<>(); - if (useRebootstrapTriggerMs) { - properties.put(CommonClientConfigs.METADATA_RECOVERY_REBOOTSTRAP_TRIGGER_MS_CONFIG, "5000"); - } else { - properties.put(CommonClientConfigs.METADATA_RECOVERY_REBOOTSTRAP_TRIGGER_MS_CONFIG, "3600000"); - properties.put(CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG, "5000"); - properties.put(CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG, "5000"); - properties.put(CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG, "1000"); - properties.put(CommonClientConfigs.RECONNECT_BACKOFF_MAX_MS_CONFIG, "1000"); + @ClusterTest( + brokers = 2, + types = {Type.KRAFT}, + serverProperties = { + @ClusterConfigProperty(key = TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, value = "true"), + @ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, value = "2") } - properties.put(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG, "rebootstrap"); - return properties; - } - - private static ClusterConfig buildConfig(Map<String, String> serverProperties, Map<String, String> rebootstrapProperties) { - return ClusterConfig.defaultBuilder() - .setTypes(Set.of(Type.KRAFT)) - .setBrokers(BROKER_COUNT) - .setServerProperties(serverProperties).build(); - } - - @ClusterTemplate(value = "generator") + ) public void testRebootstrap(ClusterInstance clusterInstance) throws InterruptedException { - var topic = "topic"; + var broker0 = 0; + var broker1 = 1; var timeout = 5; - try (var admin = clusterInstance.admin()) { - admin.createTopics(List.of(new NewTopic(topic, BROKER_COUNT, (short) 2))); - var server0 = clusterInstance.brokers().get(0); - var server1 = clusterInstance.brokers().get(1); + clusterInstance.shutdownBroker(broker0); - server1.shutdown(); - server1.awaitShutdown(); + try (var admin = clusterInstance.admin()) { + admin.createTopics(List.of(new NewTopic(TOPIC, PARTITIONS, (short) 2))); - // Only the server 0 is available for the admin client during the bootstrap. - TestUtils.waitForCondition(() -> admin.listTopics().names().get(timeout, TimeUnit.MINUTES).contains(topic), + // Only the broker 1 is available for the admin client during the bootstrap. + TestUtils.waitForCondition(() -> admin.listTopics().names().get(timeout, TimeUnit.MINUTES).contains(TOPIC), "timed out waiting for topics"); - server0.shutdown(); - server0.awaitShutdown(); - server1.startup(); + clusterInstance.shutdownBroker(broker1); + clusterInstance.startBroker(broker0); - // The server 0, originally cached during the bootstrap, is offline. - // However, the server 1 from the bootstrap list is online. + // The broker 1, originally cached during the bootstrap, is offline. + // However, the broker 0 from the bootstrap list is online. // Should be able to list topics again. - TestUtils.waitForCondition(() -> admin.listTopics().names().get(timeout, TimeUnit.MINUTES).contains(topic), + TestUtils.waitForCondition(() -> admin.listTopics().names().get(timeout, TimeUnit.MINUTES).contains(TOPIC), "timed out waiting for topics"); + } + } - server1.shutdown(); - server1.awaitShutdown(); - server0.startup(); - - // The same situation, but the server 1 has gone and server 0 is back. - TestUtils.waitForCondition(() -> admin.listTopics().names().get(timeout, TimeUnit.MINUTES).contains(topic), - "timed out waiting for topics"); + @ClusterTest( + brokers = 2, + types = {Type.KRAFT}, + serverProperties = { + @ClusterConfigProperty(key = TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, value = "true"), + @ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, value = "2") } + ) + public void testRebootstrapDisabled(ClusterInstance clusterInstance) throws InterruptedException { + var broker0 = 0; + var broker1 = 1; + + clusterInstance.shutdownBroker(broker0); + + var admin = clusterInstance.admin(Map.of(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG, "none")); + admin.createTopics(List.of(new NewTopic(TOPIC, PARTITIONS, (short) 2))); + + // Only the broker 1 is available for the admin client during the bootstrap. + TestUtils.waitForCondition(() -> admin.listTopics().names().get(5, TimeUnit.MINUTES).contains(TOPIC), + "timed out waiting for topics"); + + clusterInstance.shutdownBroker(broker1); + clusterInstance.startBroker(broker0); + + // The broker 1, originally cached during the bootstrap, is offline. + // The admin client will throw a TimeoutException because the brokers are offline during the bootstrap list Review Comment: Thank you for your comments. Sure, this situation refers to the brokers being offline since they were cached during the bootstrap. I updated the code comments. -- 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