-
Notifications
You must be signed in to change notification settings - Fork 14.3k
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
chia7712
merged 9 commits into
apache:trunk
from
frankvicky:KAFKA-19042-consumer-topic-creation
Mar 31, 2025
Merged
Changes from 8 commits
Commits
Show all changes
9 commits
Select commit
Hold shift + click to select a range
2434687
KAFKA-19042: Move ConsumerTopicCreationTest to client-integration-tes…
frankvicky 500793b
extend pollTimeout
frankvicky 4bd841e
Merge branch 'trunk' into KAFKA-19042-consumer-topic-creation
frankvicky 6023680
Merge branch 'trunk' into KAFKA-19042-consumer-topic-creation
frankvicky 5428ed8
utilize clusterInstance.consumer()
frankvicky e848ca4
refactor
frankvicky f802f24
remove unnecessary config
frankvicky 1807855
apply spotless
frankvicky f024de4
Use admin instead of broker
frankvicky File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
121 changes: 121 additions & 0 deletions
121
...tion-tests/src/test/java/org/apache/kafka/clients/consumer/ConsumerTopicCreationTest.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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(); | ||
} | ||
|
||
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); | ||
} | ||
} |
108 changes: 0 additions & 108 deletions
108
core/src/test/scala/integration/kafka/api/ConsumerTopicCreationTest.scala
This file was deleted.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
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?