Skip to content

KAFKA-19042: [1/N] Move ConsumerTopicCreationTest to client-integration-tests module #19283

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

Merged
merged 9 commits into from
Mar 31, 2025
Original file line number Diff line number Diff line change
@@ -0,0 +1,121 @@
/*
* 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.consumer;

import org.apache.kafka.common.test.ClusterInstance;
import org.apache.kafka.common.test.api.ClusterConfig;
import org.apache.kafka.common.test.api.ClusterTemplate;

import java.time.Duration;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Set;

import static org.apache.kafka.clients.consumer.ConsumerConfig.ALLOW_AUTO_CREATE_TOPICS_CONFIG;
import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_PROTOCOL_CONFIG;
import static org.apache.kafka.common.test.api.Type.KRAFT;
import static org.apache.kafka.server.config.ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;

public class ConsumerTopicCreationTest {
private static final String TOPIC = "topic";
private static final long POLL_TIMEOUT = 1000;

@ClusterTemplate("autoCreateTopicsConfigs")
void testAsyncConsumerTopicCreationIfConsumerAllowToCreateTopic(ClusterInstance cluster) {
try (Consumer<byte[], byte[]> consumer = createConsumer(cluster, GroupProtocol.CONSUMER, true)) {
subscribeAndPoll(consumer);
assertTopicCreateBasedOnPermission(cluster);
}
}

@ClusterTemplate("autoCreateTopicsConfigs")
void testAsyncConsumerTopicCreationIfConsumerDisallowToCreateTopic(ClusterInstance cluster) {
try (Consumer<byte[], byte[]> consumer = createConsumer(cluster, GroupProtocol.CONSUMER, false)) {
subscribeAndPoll(consumer);
assertTopicNotCreate(cluster);
}
}

@ClusterTemplate("autoCreateTopicsConfigs")
void testClassicConsumerTopicCreationIfConsumerAllowToCreateTopic(ClusterInstance cluster) {
try (Consumer<byte[], byte[]> consumer = createConsumer(cluster, GroupProtocol.CLASSIC, true)) {
subscribeAndPoll(consumer);
assertTopicCreateBasedOnPermission(cluster);
}
}

@ClusterTemplate("autoCreateTopicsConfigs")
void testClassicConsumerTopicCreationIfConsumerDisallowToCreateTopic(ClusterInstance cluster) {
try (Consumer<byte[], byte[]> consumer = createConsumer(cluster, GroupProtocol.CLASSIC, false)) {
subscribeAndPoll(consumer);
assertTopicNotCreate(cluster);
}
}

private void subscribeAndPoll(Consumer<byte[], byte[]> consumer) {
consumer.subscribe(List.of(TOPIC));
consumer.poll(Duration.ofMillis(POLL_TIMEOUT));
}

private void assertTopicCreateBasedOnPermission(ClusterInstance cluster) {
if (allowAutoCreateTopics(cluster))
assertTopicCreate(cluster);
else
assertTopicNotCreate(cluster);
}

private boolean allowAutoCreateTopics(ClusterInstance cluster) {
return cluster.config().serverProperties().get(AUTO_CREATE_TOPICS_ENABLE_CONFIG).equals("true");
}

private void assertTopicCreate(ClusterInstance cluster) {
assertTrue(getAllTopics(cluster).contains(TOPIC));
}

private void assertTopicNotCreate(ClusterInstance cluster) {
assertFalse(getAllTopics(cluster).contains(TOPIC),
"Both " + AUTO_CREATE_TOPICS_ENABLE_CONFIG + " and " + ALLOW_AUTO_CREATE_TOPICS_CONFIG + " need to be true to create topic automatically");
}

private Set<String> getAllTopics(ClusterInstance cluster) {
return cluster.brokers().values().iterator().next().metadataCache().getAllTopics();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you please avoid accessing broker? Maybe we can use Admin instead?

}

private static List<ClusterConfig> autoCreateTopicsConfigs() {
return List.of(
ClusterConfig.defaultBuilder()
.setTypes(Set.of(KRAFT))
.setServerProperties(Map.of(AUTO_CREATE_TOPICS_ENABLE_CONFIG, "true"))
.build(),
ClusterConfig.defaultBuilder()
.setTypes(Set.of(KRAFT))
.setServerProperties(Map.of(AUTO_CREATE_TOPICS_ENABLE_CONFIG, "false"))
.build()
);
}

private Consumer<byte[], byte[]> createConsumer(ClusterInstance cluster, GroupProtocol protocol, boolean allowConsumerAutoCreateTopics) {
Map<String, Object> consumerConfig = Map.of(
ALLOW_AUTO_CREATE_TOPICS_CONFIG, allowConsumerAutoCreateTopics,
GROUP_PROTOCOL_CONFIG, protocol.name().toLowerCase(Locale.ROOT)
);
return cluster.consumer(consumerConfig);
}
}

This file was deleted.